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 2022/02/20 17:21:34 UTC

[GitHub] [pinot] yupeng9 opened a new pull request #8233: Add Flink Pinot connector

yupeng9 opened a new pull request #8233:
URL: https://github.com/apache/pinot/pull/8233


   ## Description
   Add a Flink connector to write data to Pinot directly. This is useful for backfilling or bootstrapping tables,
   including the upsert tables. You can read more about the motivation and design in this [design proposal](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=177045634).
   
   ## Upgrade Notes
   - New plugins of Flink connector added 
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] npawar commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
npawar commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r818253261



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/PinotSinkFunction.java
##########
@@ -0,0 +1,152 @@
+/**
+ * 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.connector.flink.sink;
+
+import java.net.URI;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.pinot.connector.flink.common.RecordConverter;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.ingestion.segment.uploader.SegmentUploader;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The sink function for Pinot.
+ *
+ * @param <T> type of record supported
+ */
+@SuppressWarnings("NullAway")
+public class PinotSinkFunction<T> extends RichSinkFunction<T> implements CheckpointedFunction {
+
+  public static final long DEFAULT_SEGMENT_FLUSH_MAX_NUM_RECORDS = 500000;
+  public static final int DEFAULT_EXECUTOR_POOL_SIZE = 5;
+  public static final long DEFAULT_EXECUTOR_SHUTDOWN_WAIT_MS = 3000;
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(PinotSinkFunction.class);
+
+  private final long _segmentFlushMaxNumRecords;
+  private final int _executorPoolSize;
+
+  private final RecordConverter<T> _recordConverter;
+
+  private TableConfig _tableConfig;
+  private Schema _schema;
+
+  private transient SegmentWriter _segmentWriter;
+  private transient SegmentUploader _segmentUploader;
+  private transient ExecutorService _executor;
+  private transient long _segmentNumRecord;
+
+  public PinotSinkFunction(RecordConverter<T> recordConverter, TableConfig tableConfig, Schema schema) {
+    this(recordConverter, tableConfig, schema, DEFAULT_SEGMENT_FLUSH_MAX_NUM_RECORDS, DEFAULT_EXECUTOR_POOL_SIZE);
+  }
+
+  public PinotSinkFunction(RecordConverter<T> recordConverter, TableConfig tableConfig, Schema schema,
+      long segmentFlushMaxNumRecords, int executorPoolSize) {
+    _recordConverter = recordConverter;
+    _tableConfig = tableConfig;
+    _schema = schema;
+    _segmentFlushMaxNumRecords = segmentFlushMaxNumRecords;
+    _executorPoolSize = executorPoolSize;
+  }
+
+  @Override
+  public void open(Configuration parameters)
+      throws Exception {
+    int indexOfSubtask = this.getRuntimeContext().getIndexOfThisSubtask();
+    _segmentWriter = new FlinkSegmentWriter(indexOfSubtask, getRuntimeContext().getMetricGroup());
+    _segmentWriter.init(_tableConfig, _schema);
+    _segmentUploader = new FlinkSegmentUploader();
+    _segmentUploader.init(_tableConfig);
+    _segmentNumRecord = 0;
+    _executor = Executors.newFixedThreadPool(_executorPoolSize);
+    LOG.info("Open Pinot Sink with the table {}", _tableConfig.toJsonString());
+  }
+
+  @Override
+  public void close()
+      throws Exception {
+    LOG.info("Closing Pinot Sink");
+    try {
+      if (_segmentNumRecord > 0) {
+        flush();
+      }
+    } catch (Exception e) {
+      LOG.error("Error when closing Pinot sink", e);
+    }
+    _executor.shutdown();
+    try {
+      if (!_executor.awaitTermination(DEFAULT_EXECUTOR_SHUTDOWN_WAIT_MS, TimeUnit.MILLISECONDS)) {
+        _executor.shutdownNow();
+      }
+    } catch (InterruptedException e) {
+      _executor.shutdownNow();
+    }
+    _segmentWriter.close();

Review comment:
       do this in finally?

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/common/PinotRowRecordConverter.java
##########
@@ -0,0 +1,58 @@
+/**
+ * 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.connector.flink.common;
+
+import java.util.Map;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.types.Row;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/** Converts {@link Row} type data into {@link GenericRow} format. */
+public class PinotRowRecordConverter implements RecordConverter<Row> {
+
+  private final RowTypeInfo _rowTypeInfo;
+  private final String[] _fieldNames;
+  private final TypeInformation<?>[] _fieldTypes;
+
+  public PinotRowRecordConverter(RowTypeInfo rowTypeInfo) {
+    _rowTypeInfo = rowTypeInfo;
+    _fieldNames = rowTypeInfo.getFieldNames();
+    _fieldTypes = rowTypeInfo.getFieldTypes();

Review comment:
       this is unused. remove?

##########
File path: pinot-connectors/pinot-flink-connector/README.md
##########
@@ -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.
+
+-->
+# Flink-Pinot Connector
+
+Flink connector to write data to Pinot directly. This is useful for backfilling or bootstrapping tables,
+including the upsert tables. You can read more about the motivation and design in this [design proposal](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=177045634).
+
+## Quick Start
+```java
+StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+execEnv.setParallelism(2);
+DataStream<Row> srcDs = execEnv.fromCollection(data).returns(TEST_TYPE_INFO).keyBy(r -> r.getField(0));
+
+PinotControllerClient client = new PinotControllerClient();
+// fetch Pinot schema
+Schema schema = PinotConnectionUtils.getSchema(client, "starbucksStores");
+// fetch Pinot table config
+TableConfig tableConfig = PinotConnectionUtils.getTableConfig(client, "starbucksStores", "OFFLINE");
+// create Flink Pinot Sink
+srcDs.addSink(new PinotSinkFunction<>(new PinotRowRecordConverter(TEST_TYPE_INFO), tableConfig, schema));
+execEnv.execute();
+```
+
+For more examples, please see `src/main/java/org/apache/pinot/connector/flink/FlinkQuickStart.java`
+
+## Notes for backfilling upsert table
+ - To correctly partition the output segments by the primary key, the Flink job *must* also include the partitionBeyKey operator before the Sink operator

Review comment:
       typo s/partitionBeyKey/partitionByKey

##########
File path: pinot-connectors/pinot-flink-connector/README.md
##########
@@ -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.
+
+-->
+# Flink-Pinot Connector
+
+Flink connector to write data to Pinot directly. This is useful for backfilling or bootstrapping tables,
+including the upsert tables. You can read more about the motivation and design in this [design proposal](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=177045634).
+
+## Quick Start
+```java
+StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+execEnv.setParallelism(2);
+DataStream<Row> srcDs = execEnv.fromCollection(data).returns(TEST_TYPE_INFO).keyBy(r -> r.getField(0));
+
+PinotControllerClient client = new PinotControllerClient();
+// fetch Pinot schema
+Schema schema = PinotConnectionUtils.getSchema(client, "starbucksStores");
+// fetch Pinot table config
+TableConfig tableConfig = PinotConnectionUtils.getTableConfig(client, "starbucksStores", "OFFLINE");
+// create Flink Pinot Sink
+srcDs.addSink(new PinotSinkFunction<>(new PinotRowRecordConverter(TEST_TYPE_INFO), tableConfig, schema));
+execEnv.execute();
+```
+
+For more examples, please see `src/main/java/org/apache/pinot/connector/flink/FlinkQuickStart.java`
+
+## Notes for backfilling upsert table
+ - To correctly partition the output segments by the primary key, the Flink job *must* also include the partitionBeyKey operator before the Sink operator
+ - The parallelism of the job *must* be set the same as the number of partitions of the Pinot table, so that the sink in each task executor can generate the segment of same partitions.
+ - It’s important to plan the resource usage to avoid capacity issues such as out of memory. In particular, Pinot sink has an in-memory buffer of records, and it flushes when the threshold is reached. Currently, the threshold on the number of records is supported via the config of `segmentFlushMaxNumRecords`. In the future, we could add other types of threshold such as the memory usage of the buffer.

Review comment:
       it's not an in-memory buffer right? collect writes each record to file. the capacity issue to call out is out of disk space

##########
File path: pinot-connectors/pinot-flink-connector/src/test/java/org/apache/pinot/connector/flink/util/TestUtils.java
##########
@@ -0,0 +1,44 @@
+/**
+ * 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.connector.flink.util;
+
+import java.io.File;
+import java.io.IOException;
+import javax.annotation.Nonnull;
+import org.apache.commons.io.FileUtils;
+import org.testng.Assert;
+
+
+public final class TestUtils {

Review comment:
       can the integration test move to pinot-integration-tests? 1. it should ideally be there with all others 2. you won't have to write this util class just for flink connector, as it's already there for use in ITs

##########
File path: pinot-connectors/pinot-flink-connector/src/test/resources/fixtures/pinotTableSchema.json
##########
@@ -0,0 +1,59 @@
+{
+  "schemaName": "demand",
+  "dimensionFieldSpecs": [
+    {
+      "name": "demand_uuid",
+      "singleValueField": true,
+      "dataType": "STRING",
+      "virtualColumnProvider": null,
+      "defaultNullValue": "null",
+      "transformFunction": null,
+      "maxLength" : 4096
+    },
+    {
+      "name": "geofence",
+      "singleValueField": true,
+      "dataType": "STRING",
+      "virtualColumnProvider": null,
+      "defaultNullValue": "null",
+      "transformFunction": null
+    }
+  ],
+  "metricFieldSpecs": [
+    {
+      "dataType": "DOUBLE",
+      "singleValueField": true,
+      "fieldSize": 8,
+      "derivedMetricType": null,
+      "name": "surge_multiplier",
+      "virtualColumnProvider": null,
+      "defaultNullValue": 0.0,
+      "transformFunction": null
+    }
+  ],
+  "dateTimeFieldSpecs": [
+  ],
+  "timeFieldSpec": {

Review comment:
       nit: use the dateTimeFieldSpec. often our oss users look at the quickstarts and copy the configs. we don't want anyone to be using timeFieldSpec anymore

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/FlinkQuickStart.java
##########
@@ -0,0 +1,88 @@
+/**
+ * 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.connector.flink;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.pinot.connector.flink.common.PinotRowRecordConverter;
+import org.apache.pinot.connector.flink.http.PinotConnectionUtils;
+import org.apache.pinot.connector.flink.http.PinotControllerClient;
+import org.apache.pinot.connector.flink.sink.PinotSinkFunction;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+
+
+/**
+ * A quick start to populate a segment into Pinot Table using the connector. Please run the GenericQuickStart to create
+ * the offline table of all Starbucks store locations in US, and then run this quick start to populate other Starbucks
+ * stores in the rest of the world.
+ */
+public final class FlinkQuickStart {

Review comment:
       can this just be in pinot-tools with the rest of the Quickstarts?

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/common/PinotMapRecordConverter.java
##########
@@ -0,0 +1,41 @@
+/**
+ * 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.connector.flink.common;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+public class PinotMapRecordConverter implements RecordConverter<Map<String, Object>> {

Review comment:
       (optional) this name is not the very intuitive..
   How about PinotGenericRowConverter as interface, and MapGenericRowConverter and FlinkRowGenericRowConverter as impls?

##########
File path: pinot-connectors/pinot-flink-connector/src/test/java/org/apache/pinot/connector/flink/util/TestUtils.java
##########
@@ -0,0 +1,44 @@
+/**
+ * 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.connector.flink.util;
+
+import java.io.File;
+import java.io.IOException;
+import javax.annotation.Nonnull;
+import org.apache.commons.io.FileUtils;
+import org.testng.Assert;
+
+
+public final class TestUtils {
+
+  private TestUtils() {
+  }
+
+  public static void ensureDirectoriesExistAndEmpty(@Nonnull File... dirs)
+      throws IOException {
+    File[] var1 = dirs;
+    int var2 = dirs.length;
+
+    for (int var3 = 0; var3 < var2; var3++) {

Review comment:
       if you do decide to keep this class, change the var names




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6eef992) into [master](https://codecov.io/gh/apache/pinot/commit/46ed731c4e60c308c9559e46349a984b0ce05ce6?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (46ed731) will **decrease** coverage by `3.82%`.
   > The diff coverage is `n/a`.
   
   > :exclamation: Current head 6eef992 differs from pull request most recent head 05d2c8e. Consider uploading reports for the commit 05d2c8e to get more accurate results
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     70.83%   67.00%   -3.83%     
   + Complexity     4245     4166      -79     
   ============================================
     Files          1631     1231     -400     
     Lines         85462    62258   -23204     
     Branches      12877     9699    -3178     
   ============================================
   - Hits          60539    41718   -18821     
   + Misses        20746    17575    -3171     
   + Partials       4177     2965    -1212     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `67.00% <ø> (+0.04%)` | :arrow_up: |
   | unittests2 | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ache/pinot/server/access/AccessControlFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VydmVyL2FjY2Vzcy9BY2Nlc3NDb250cm9sRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/SegmentReloadMessage.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvU2VnbWVudFJlbG9hZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/core/data/manager/realtime/TimerService.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvVGltZXJTZXJ2aWNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...not/common/exception/HttpErrorStatusException.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL0h0dHBFcnJvclN0YXR1c0V4Y2VwdGlvbi5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...t/core/plan/StreamingInstanceResponsePlanNode.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9wbGFuL1N0cmVhbWluZ0luc3RhbmNlUmVzcG9uc2VQbGFuTm9kZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...t/core/startree/plan/StarTreeDocIdSetPlanNode.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9zdGFydHJlZS9wbGFuL1N0YXJUcmVlRG9jSWRTZXRQbGFuTm9kZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [620 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [46ed731...05d2c8e](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r820370658



##########
File path: pinot-connectors/pinot-flink-connector/src/test/java/org/apache/pinot/connector/flink/util/TestUtils.java
##########
@@ -0,0 +1,44 @@
+/**
+ * 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.connector.flink.util;
+
+import java.io.File;
+import java.io.IOException;
+import javax.annotation.Nonnull;
+import org.apache.commons.io.FileUtils;
+import org.testng.Assert;
+
+
+public final class TestUtils {

Review comment:
       hmm, the integration test in the spark connector is also in the spark module. I'm fine with the suggestion, though it'll make connectors build non-optional in Pinot project build. What do you think, @xiangfu0 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r813099492



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/PinotControllerClient.java
##########
@@ -0,0 +1,147 @@
+/**
+ * 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.connector.flink.http;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonValue;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/** This file is partially copied from RTAUMS gateway package */
+public class PinotControllerClient extends HttpClient {

Review comment:
       yup, I'm fine with consolidating/extracting the client into a more reusable lib. But I think it's better to do this in a later PR, to reduce the scope of this PR, which is already quite large.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2a55ab2) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `0.05%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     71.06%   71.00%   -0.06%     
   + Complexity     4320     4319       -1     
   ============================================
     Files          1626     1626              
     Lines         85036    85076      +40     
     Branches      12795    12802       +7     
   ============================================
   - Hits          60428    60408      -20     
   - Misses        20456    20513      +57     
   - Partials       4152     4155       +3     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `28.76% <ø> (-0.09%)` | :arrow_down: |
   | integration2 | `27.45% <ø> (-0.09%)` | :arrow_down: |
   | unittests1 | `67.35% <ø> (+<0.01%)` | :arrow_up: |
   | unittests2 | `14.12% <ø> (+0.04%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...data/manager/realtime/DefaultSegmentCommitter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvRGVmYXVsdFNlZ21lbnRDb21taXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-80.00%)` | :arrow_down: |
   | [...er/api/resources/LLCSegmentCompletionHandlers.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL0xMQ1NlZ21lbnRDb21wbGV0aW9uSGFuZGxlcnMuamF2YQ==) | `43.56% <0.00%> (-18.82%)` | :arrow_down: |
   | [...data/manager/realtime/SegmentCommitterFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvU2VnbWVudENvbW1pdHRlckZhY3RvcnkuamF2YQ==) | `88.23% <0.00%> (-11.77%)` | :arrow_down: |
   | [...pinot/common/utils/fetcher/HttpSegmentFetcher.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvZmV0Y2hlci9IdHRwU2VnbWVudEZldGNoZXIuamF2YQ==) | `61.53% <0.00%> (-10.26%)` | :arrow_down: |
   | [...inot/core/util/SegmentCompletionProtocolUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL1NlZ21lbnRDb21wbGV0aW9uUHJvdG9jb2xVdGlscy5qYXZh) | `57.69% <0.00%> (-7.70%)` | :arrow_down: |
   | [...altime/ServerSegmentCompletionProtocolHandler.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VydmVyL3JlYWx0aW1lL1NlcnZlclNlZ21lbnRDb21wbGV0aW9uUHJvdG9jb2xIYW5kbGVyLmphdmE=) | `51.42% <0.00%> (-6.67%)` | :arrow_down: |
   | [...nMaxValueBasedSelectionOrderByCombineOperator.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9jb21iaW5lL01pbk1heFZhbHVlQmFzZWRTZWxlY3Rpb25PcmRlckJ5Q29tYmluZU9wZXJhdG9yLmphdmE=) | `70.45% <0.00%> (-2.28%)` | :arrow_down: |
   | [...r/helix/SegmentOnlineOfflineStateModelFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VydmVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zZXJ2ZXIvc3RhcnRlci9oZWxpeC9TZWdtZW50T25saW5lT2ZmbGluZVN0YXRlTW9kZWxGYWN0b3J5LmphdmE=) | `63.20% <0.00%> (-1.89%)` | :arrow_down: |
   | [...e/pinot/segment/local/io/util/PinotDataBitSet.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pby91dGlsL1Bpbm90RGF0YUJpdFNldC5qYXZh) | `95.62% <0.00%> (-1.46%)` | :arrow_down: |
   | [...ces/PinotSegmentUploadDownloadRestletResource.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL1Bpbm90U2VnbWVudFVwbG9hZERvd25sb2FkUmVzdGxldFJlc291cmNlLmphdmE=) | `57.51% <0.00%> (-1.29%)` | :arrow_down: |
   | ... and [21 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...2a55ab2](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (657daa5) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `0.38%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     71.06%   70.67%   -0.39%     
   + Complexity     4320     4240      -80     
   ============================================
     Files          1626     1631       +5     
     Lines         85036    85281     +245     
     Branches      12795    12845      +50     
   ============================================
   - Hits          60428    60273     -155     
   - Misses        20456    20839     +383     
   - Partials       4152     4169      +17     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `28.73% <ø> (-0.12%)` | :arrow_down: |
   | integration2 | `27.52% <ø> (-0.02%)` | :arrow_down: |
   | unittests1 | `66.96% <ø> (-0.39%)` | :arrow_down: |
   | unittests2 | `14.10% <ø> (+0.03%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...data/manager/realtime/DefaultSegmentCommitter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvRGVmYXVsdFNlZ21lbnRDb21taXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-80.00%)` | :arrow_down: |
   | [...ls/nativefst/automaton/MinimizationOperations.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL01pbmltaXphdGlvbk9wZXJhdGlvbnMuamF2YQ==) | `0.00% <0.00%> (-36.20%)` | :arrow_down: |
   | [...tils/nativefst/automaton/TransitionComparator.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL1RyYW5zaXRpb25Db21wYXJhdG9yLmphdmE=) | `9.37% <0.00%> (-25.00%)` | :arrow_down: |
   | [...ller/helix/core/minion/TaskTypeMetricsUpdater.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL21pbmlvbi9UYXNrVHlwZU1ldHJpY3NVcGRhdGVyLmphdmE=) | `80.00% <0.00%> (-20.00%)` | :arrow_down: |
   | [...er/api/resources/LLCSegmentCompletionHandlers.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL0xMQ1NlZ21lbnRDb21wbGV0aW9uSGFuZGxlcnMuamF2YQ==) | `43.56% <0.00%> (-18.82%)` | :arrow_down: |
   | [...ent/local/utils/nativefst/automaton/Automaton.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL0F1dG9tYXRvbi5qYXZh) | `57.08% <0.00%> (-16.86%)` | :arrow_down: |
   | [...data/manager/realtime/SegmentCommitterFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvU2VnbWVudENvbW1pdHRlckZhY3RvcnkuamF2YQ==) | `88.23% <0.00%> (-11.77%)` | :arrow_down: |
   | [...gment/spi/partition/HashCodePartitionFunction.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1zcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3NlZ21lbnQvc3BpL3BhcnRpdGlvbi9IYXNoQ29kZVBhcnRpdGlvbkZ1bmN0aW9uLmphdmE=) | `66.66% <0.00%> (-11.12%)` | :arrow_down: |
   | [...cal/utils/nativefst/automaton/BasicOperations.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL0Jhc2ljT3BlcmF0aW9ucy5qYXZh) | `28.57% <0.00%> (-10.85%)` | :arrow_down: |
   | [...pinot/common/utils/fetcher/HttpSegmentFetcher.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvZmV0Y2hlci9IdHRwU2VnbWVudEZldGNoZXIuamF2YQ==) | `61.53% <0.00%> (-10.26%)` | :arrow_down: |
   | ... and [85 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...657daa5](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d68d991) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `6.99%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     71.06%   64.06%   -7.00%     
   + Complexity     4320     4242      -78     
   ============================================
     Files          1626     1586      -40     
     Lines         85036    83399    -1637     
     Branches      12795    12641     -154     
   ============================================
   - Hits          60428    53431    -6997     
   - Misses        20456    26120    +5664     
   + Partials       4152     3848     -304     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `66.98% <ø> (-0.37%)` | :arrow_down: |
   | unittests2 | `14.12% <ø> (+0.04%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ache/pinot/server/access/AccessControlFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VydmVyL2FjY2Vzcy9BY2Nlc3NDb250cm9sRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/SegmentReloadMessage.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvU2VnbWVudFJlbG9hZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/core/data/manager/realtime/TimerService.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvVGltZXJTZXJ2aWNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...che/pinot/controller/util/TableMetadataReader.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci91dGlsL1RhYmxlTWV0YWRhdGFSZWFkZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/minion/exception/TaskCancelledException.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vZXhjZXB0aW9uL1Rhc2tDYW5jZWxsZWRFeGNlcHRpb24uamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...not/common/exception/HttpErrorStatusException.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL0h0dHBFcnJvclN0YXR1c0V4Y2VwdGlvbi5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [420 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...d68d991](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d68d991) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `56.93%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #8233       +/-   ##
   =============================================
   - Coverage     71.06%   14.12%   -56.94%     
   + Complexity     4320       81     -4239     
   =============================================
     Files          1626     1586       -40     
     Lines         85036    83399     -1637     
     Branches      12795    12641      -154     
   =============================================
   - Hits          60428    11780    -48648     
   - Misses        20456    70735    +50279     
   + Partials       4152      884     -3268     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `14.12% <ø> (+0.04%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ain/java/org/apache/pinot/core/data/table/Key.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL0tleS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/spi/utils/BooleanUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQm9vbGVhblV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/data/table/Record.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1JlY29yZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/util/GroupByUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dyb3VwQnlVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/config/table/FSTType.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL0ZTVFR5cGUuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/data/MetricFieldSpec.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9NZXRyaWNGaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/spi/utils/BigDecimalUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQmlnRGVjaW1hbFV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/common/tier/TierFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdGllci9UaWVyRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/common/utils/StringUtil.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvU3RyaW5nVXRpbC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/core/data/table/BaseTable.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL0Jhc2VUYWJsZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1307 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...d68d991](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] xiangfu0 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r814287411



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/PinotControllerClient.java
##########
@@ -0,0 +1,147 @@
+/**
+ * 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.connector.flink.http;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonValue;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/** This file is partially copied from RTAUMS gateway package */
+public class PinotControllerClient extends HttpClient {

Review comment:
       Sounds good, can you create a new issue for consolidating the reusable utils with potential candidates that can be merged.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] npawar commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
npawar commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r818240780



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,266 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a buffer. The {@link GenericRow} are written to
+ * the buffer as AVRO records.
+ */
+@SuppressWarnings("NullAway")
+@NotThreadSafe
+public class FlinkSegmentWriter implements SegmentWriter {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FlinkSegmentWriter.class);
+  private static final FileFormat BUFFER_FILE_FORMAT = FileFormat.AVRO;
+
+  private final int _indexOfSubtask;
+
+  private TableConfig _tableConfig;
+  private String _tableNameWithType;
+  private BatchIngestionConfig _batchIngestionConfig;
+  private BatchConfig _batchConfig;
+  private String _outputDirURI;
+  private Schema _schema;
+  private Set<String> _fieldsToRead;
+  private RecordTransformer _recordTransformer;
+
+  private File _stagingDir;
+  private File _bufferFile;
+  private int _rowCount;
+  /** A sequence ID that increments each time a segment is flushed */
+  private int _seqId;
+
+  private org.apache.avro.Schema _avroSchema;
+  private DataFileWriter<GenericData.Record> _recordWriter;
+  private GenericData.Record _reusableRecord;
+
+  // metrics
+  private transient Counter _processedRecords;
+  private transient volatile long _lastRecordProcessingTimeMs = 0;
+
+  public FlinkSegmentWriter(int indexOfSubtask, MetricGroup metricGrp) {
+    _indexOfSubtask = indexOfSubtask;
+    registerMetrics(metricGrp);
+  }
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema)
+      throws Exception {
+    init(tableConfig, schema, null);
+  }
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema, Map<String, String> batchConfigOverride)
+      throws Exception {
+    _rowCount = 0;
+    _seqId = 0;
+    _tableConfig = tableConfig;
+    _tableNameWithType = _tableConfig.getTableName();
+    Preconditions.checkState(
+        _tableConfig.getIngestionConfig() != null && _tableConfig.getIngestionConfig().getBatchIngestionConfig() != null
+            && CollectionUtils.isNotEmpty(
+            _tableConfig.getIngestionConfig().getBatchIngestionConfig().getBatchConfigMaps()),
+        "Must provide ingestionConfig->batchIngestionConfig->batchConfigMaps in tableConfig for table: %s",
+        _tableNameWithType);
+    _batchIngestionConfig = _tableConfig.getIngestionConfig().getBatchIngestionConfig();
+    Preconditions.checkState(_batchIngestionConfig.getBatchConfigMaps().size() == 1,
+        "batchConfigMaps must contain only 1 BatchConfig for table: %s", _tableNameWithType);
+
+    Map<String, String> batchConfigMap = _batchIngestionConfig.getBatchConfigMaps().get(0);
+    String segmentNamePostfixProp = String.format("%s.%s", BatchConfigProperties.SEGMENT_NAME_GENERATOR_PROP_PREFIX,
+        BatchConfigProperties.SEGMENT_NAME_POSTFIX);
+    String segmentSuffix = batchConfigMap.get(segmentNamePostfixProp);
+    segmentSuffix = segmentSuffix == null ? String.valueOf(_indexOfSubtask) : segmentSuffix + "_" + _indexOfSubtask;
+    batchConfigMap.put(segmentNamePostfixProp, segmentSuffix);
+
+    _batchConfig = new BatchConfig(_tableNameWithType, batchConfigMap);
+
+    Preconditions.checkState(StringUtils.isNotBlank(_batchConfig.getOutputDirURI()),
+        "Must provide: %s in batchConfigs for table: %s", BatchConfigProperties.OUTPUT_DIR_URI, _tableNameWithType);
+    _outputDirURI = _batchConfig.getOutputDirURI();
+    Files.createDirectories(Paths.get(_outputDirURI));
+
+    _schema = schema;
+    _fieldsToRead = _schema.getColumnNames();
+    _recordTransformer = CompositeTransformer.getDefaultTransformer(_tableConfig, _schema);
+    _avroSchema = SegmentProcessorAvroUtils.convertPinotSchemaToAvroSchema(_schema);
+    _reusableRecord = new GenericData.Record(_avroSchema);
+
+    // Create tmp dir
+    _stagingDir = new File(FileUtils.getTempDirectory(),
+        String.format("segment_writer_staging_%s_%d_%d", _tableNameWithType, _indexOfSubtask,
+            System.currentTimeMillis()));
+    Preconditions.checkState(_stagingDir.mkdirs(), "Failed to create staging dir: %s", _stagingDir.getAbsolutePath());
+
+    // Create buffer file
+    File bufferDir = new File(_stagingDir, "buffer_dir");
+    Preconditions.checkState(bufferDir.mkdirs(), "Failed to create buffer_dir: %s", bufferDir.getAbsolutePath());
+    _bufferFile = new File(bufferDir, "buffer_file");
+    resetBuffer();
+    LOGGER.info("Initialized {} for Pinot table: {}", this.getClass().getName(), _tableNameWithType);
+  }
+
+  private void registerMetrics(MetricGroup metricGrp) {
+    _processedRecords = metricGrp.counter("records.processed");
+    metricGrp.gauge("record.processing.time.ts", (Gauge<Long>) () -> _lastRecordProcessingTimeMs);
+  }
+
+  private void resetBuffer()
+      throws IOException {
+    FileUtils.deleteQuietly(_bufferFile);
+    _rowCount = 0;
+    _recordWriter = new DataFileWriter<>(new GenericDatumWriter<>(_avroSchema));
+    _recordWriter.create(_avroSchema, _bufferFile);
+  }
+
+  @Override
+  public void collect(GenericRow row)
+      throws IOException {
+    long startTime = System.currentTimeMillis();
+    GenericRow transform = _recordTransformer.transform(row);
+    SegmentProcessorAvroUtils.convertGenericRowToAvroRecord(transform, _reusableRecord, _fieldsToRead);
+    _rowCount++;
+    _recordWriter.append(_reusableRecord);
+    _lastRecordProcessingTimeMs = System.currentTimeMillis() - startTime;
+    _processedRecords.inc();
+  }
+
+  /**
+   * Creates one Pinot segment using the {@link GenericRow}s collected in the AVRO file buffer, at
+   * the outputDirUri as specified in the tableConfig->batchConfigs. Successful invocation of this
+   * method means that the {@link GenericRow}s collected so far, are now available in the Pinot
+   * segment and not available in the buffer anymore.
+   *
+   * <p>Successful completion of segment will return the segment URI, and the URI includes a

Review comment:
       an example here of where to expect the seqID will be nice

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,266 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a buffer. The {@link GenericRow} are written to
+ * the buffer as AVRO records.
+ */
+@SuppressWarnings("NullAway")
+@NotThreadSafe
+public class FlinkSegmentWriter implements SegmentWriter {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FlinkSegmentWriter.class);
+  private static final FileFormat BUFFER_FILE_FORMAT = FileFormat.AVRO;
+
+  private final int _indexOfSubtask;
+
+  private TableConfig _tableConfig;
+  private String _tableNameWithType;
+  private BatchIngestionConfig _batchIngestionConfig;
+  private BatchConfig _batchConfig;
+  private String _outputDirURI;
+  private Schema _schema;
+  private Set<String> _fieldsToRead;
+  private RecordTransformer _recordTransformer;
+
+  private File _stagingDir;
+  private File _bufferFile;
+  private int _rowCount;
+  /** A sequence ID that increments each time a segment is flushed */
+  private int _seqId;
+
+  private org.apache.avro.Schema _avroSchema;
+  private DataFileWriter<GenericData.Record> _recordWriter;
+  private GenericData.Record _reusableRecord;
+
+  // metrics
+  private transient Counter _processedRecords;
+  private transient volatile long _lastRecordProcessingTimeMs = 0;
+
+  public FlinkSegmentWriter(int indexOfSubtask, MetricGroup metricGrp) {
+    _indexOfSubtask = indexOfSubtask;
+    registerMetrics(metricGrp);
+  }
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema)
+      throws Exception {
+    init(tableConfig, schema, null);
+  }
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema, Map<String, String> batchConfigOverride)
+      throws Exception {
+    _rowCount = 0;
+    _seqId = 0;
+    _tableConfig = tableConfig;
+    _tableNameWithType = _tableConfig.getTableName();
+    Preconditions.checkState(
+        _tableConfig.getIngestionConfig() != null && _tableConfig.getIngestionConfig().getBatchIngestionConfig() != null
+            && CollectionUtils.isNotEmpty(
+            _tableConfig.getIngestionConfig().getBatchIngestionConfig().getBatchConfigMaps()),
+        "Must provide ingestionConfig->batchIngestionConfig->batchConfigMaps in tableConfig for table: %s",
+        _tableNameWithType);
+    _batchIngestionConfig = _tableConfig.getIngestionConfig().getBatchIngestionConfig();
+    Preconditions.checkState(_batchIngestionConfig.getBatchConfigMaps().size() == 1,
+        "batchConfigMaps must contain only 1 BatchConfig for table: %s", _tableNameWithType);
+
+    Map<String, String> batchConfigMap = _batchIngestionConfig.getBatchConfigMaps().get(0);
+    String segmentNamePostfixProp = String.format("%s.%s", BatchConfigProperties.SEGMENT_NAME_GENERATOR_PROP_PREFIX,
+        BatchConfigProperties.SEGMENT_NAME_POSTFIX);
+    String segmentSuffix = batchConfigMap.get(segmentNamePostfixProp);
+    segmentSuffix = segmentSuffix == null ? String.valueOf(_indexOfSubtask) : segmentSuffix + "_" + _indexOfSubtask;
+    batchConfigMap.put(segmentNamePostfixProp, segmentSuffix);
+
+    _batchConfig = new BatchConfig(_tableNameWithType, batchConfigMap);
+
+    Preconditions.checkState(StringUtils.isNotBlank(_batchConfig.getOutputDirURI()),
+        "Must provide: %s in batchConfigs for table: %s", BatchConfigProperties.OUTPUT_DIR_URI, _tableNameWithType);
+    _outputDirURI = _batchConfig.getOutputDirURI();
+    Files.createDirectories(Paths.get(_outputDirURI));
+
+    _schema = schema;
+    _fieldsToRead = _schema.getColumnNames();
+    _recordTransformer = CompositeTransformer.getDefaultTransformer(_tableConfig, _schema);
+    _avroSchema = SegmentProcessorAvroUtils.convertPinotSchemaToAvroSchema(_schema);
+    _reusableRecord = new GenericData.Record(_avroSchema);
+
+    // Create tmp dir
+    _stagingDir = new File(FileUtils.getTempDirectory(),
+        String.format("segment_writer_staging_%s_%d_%d", _tableNameWithType, _indexOfSubtask,
+            System.currentTimeMillis()));
+    Preconditions.checkState(_stagingDir.mkdirs(), "Failed to create staging dir: %s", _stagingDir.getAbsolutePath());
+
+    // Create buffer file
+    File bufferDir = new File(_stagingDir, "buffer_dir");
+    Preconditions.checkState(bufferDir.mkdirs(), "Failed to create buffer_dir: %s", bufferDir.getAbsolutePath());
+    _bufferFile = new File(bufferDir, "buffer_file");
+    resetBuffer();
+    LOGGER.info("Initialized {} for Pinot table: {}", this.getClass().getName(), _tableNameWithType);
+  }
+
+  private void registerMetrics(MetricGroup metricGrp) {
+    _processedRecords = metricGrp.counter("records.processed");
+    metricGrp.gauge("record.processing.time.ts", (Gauge<Long>) () -> _lastRecordProcessingTimeMs);
+  }
+
+  private void resetBuffer()
+      throws IOException {
+    FileUtils.deleteQuietly(_bufferFile);
+    _rowCount = 0;
+    _recordWriter = new DataFileWriter<>(new GenericDatumWriter<>(_avroSchema));
+    _recordWriter.create(_avroSchema, _bufferFile);
+  }
+
+  @Override
+  public void collect(GenericRow row)
+      throws IOException {
+    long startTime = System.currentTimeMillis();
+    GenericRow transform = _recordTransformer.transform(row);
+    SegmentProcessorAvroUtils.convertGenericRowToAvroRecord(transform, _reusableRecord, _fieldsToRead);
+    _rowCount++;
+    _recordWriter.append(_reusableRecord);
+    _lastRecordProcessingTimeMs = System.currentTimeMillis() - startTime;
+    _processedRecords.inc();
+  }
+
+  /**
+   * Creates one Pinot segment using the {@link GenericRow}s collected in the AVRO file buffer, at
+   * the outputDirUri as specified in the tableConfig->batchConfigs. Successful invocation of this
+   * method means that the {@link GenericRow}s collected so far, are now available in the Pinot
+   * segment and not available in the buffer anymore.
+   *
+   * <p>Successful completion of segment will return the segment URI, and the URI includes a
+   * sequence id indicating the part number. The sequence id is initialized to 0 and each successful
+   * flush will increment the sequence id by 1. The buffer will be reset and ready to accept further
+   * records via <code>collect()</code> If an exception is thrown, the buffer will not be reset and
+   * so, <code>flush()</code> can be invoked repeatedly in a retry loop. If a successful invocation
+   * is not achieved,<code>close()</code> followed by <code>init </code> will have to be called in
+   * order to reset the buffer and resume record writing.
+   *
+   * @return URI of the generated segment
+   * @throws IOException
+   */
+  @Override
+  public URI flush()
+      throws IOException {
+
+    LOGGER.info("Beginning flush for Pinot table: {} with {} records", _tableNameWithType, _rowCount);
+    _recordWriter.close();
+
+    // Create temp dir for flush
+    File flushDir = new File(_stagingDir, "flush_dir_" + System.currentTimeMillis());
+    Preconditions.checkState(flushDir.mkdirs(), "Failed to create flush dir: %s", flushDir);
+
+    try {
+      // Segment dir
+      File segmentDir = new File(flushDir, "segment_dir");
+
+      // Make BatchIngestionConfig for flush
+      Map<String, String> batchConfigMapOverride = new HashMap<>(_batchConfig.getBatchConfigMap());
+      batchConfigMapOverride.put(BatchConfigProperties.INPUT_DIR_URI, _bufferFile.getAbsolutePath());
+      batchConfigMapOverride.put(BatchConfigProperties.OUTPUT_DIR_URI, segmentDir.getAbsolutePath());
+      batchConfigMapOverride.put(BatchConfigProperties.INPUT_FORMAT, BUFFER_FILE_FORMAT.toString());
+      BatchIngestionConfig batchIngestionConfig = new BatchIngestionConfig(Lists.newArrayList(batchConfigMapOverride),
+          _batchIngestionConfig.getSegmentIngestionType(), _batchIngestionConfig.getSegmentIngestionFrequency());
+
+      // Build segment
+      SegmentGeneratorConfig segmentGeneratorConfig =
+          IngestionUtils.generateSegmentGeneratorConfig(_tableConfig, _schema, batchIngestionConfig);
+      segmentGeneratorConfig.setSequenceId(_seqId);
+      String segmentName = IngestionUtils.buildSegment(segmentGeneratorConfig);
+      LOGGER.info("Successfully built segment: {} of sequence {} for Pinot table: {}", segmentName, _seqId,
+          _tableNameWithType);
+
+      // Tar segment
+      File segmentTarFile =
+          new File(_outputDirURI, String.format("%s_%d%s", segmentName, _indexOfSubtask, Constants.TAR_GZ_FILE_EXT));

Review comment:
       since you added it to the segmentSuffix in BatchConfig, it will already be used by SegmentNameGenerator. Why do we need to explicitly add it in this tar name again?

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentUploader.java
##########
@@ -0,0 +1,114 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.spi.auth.AuthContext;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.filesystem.PinotFS;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.uploader.SegmentUploader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Default implementation of {@link SegmentUploader} with support for all push modes The configs for
+ * push are fetched from batchConfigMaps of tableConfig
+ */
+@SuppressWarnings("NullAway")

Review comment:
       This class seems exactly identical to SegmentUploaderDefault. Can we just use that one?

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,266 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a buffer. The {@link GenericRow} are written to
+ * the buffer as AVRO records.
+ */
+@SuppressWarnings("NullAway")
+@NotThreadSafe
+public class FlinkSegmentWriter implements SegmentWriter {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FlinkSegmentWriter.class);
+  private static final FileFormat BUFFER_FILE_FORMAT = FileFormat.AVRO;
+
+  private final int _indexOfSubtask;
+
+  private TableConfig _tableConfig;
+  private String _tableNameWithType;
+  private BatchIngestionConfig _batchIngestionConfig;
+  private BatchConfig _batchConfig;
+  private String _outputDirURI;
+  private Schema _schema;
+  private Set<String> _fieldsToRead;
+  private RecordTransformer _recordTransformer;
+
+  private File _stagingDir;
+  private File _bufferFile;
+  private int _rowCount;
+  /** A sequence ID that increments each time a segment is flushed */
+  private int _seqId;
+
+  private org.apache.avro.Schema _avroSchema;
+  private DataFileWriter<GenericData.Record> _recordWriter;
+  private GenericData.Record _reusableRecord;
+
+  // metrics
+  private transient Counter _processedRecords;
+  private transient volatile long _lastRecordProcessingTimeMs = 0;
+
+  public FlinkSegmentWriter(int indexOfSubtask, MetricGroup metricGrp) {
+    _indexOfSubtask = indexOfSubtask;
+    registerMetrics(metricGrp);
+  }
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema)
+      throws Exception {
+    init(tableConfig, schema, null);
+  }
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema, Map<String, String> batchConfigOverride)
+      throws Exception {
+    _rowCount = 0;
+    _seqId = 0;
+    _tableConfig = tableConfig;
+    _tableNameWithType = _tableConfig.getTableName();
+    Preconditions.checkState(
+        _tableConfig.getIngestionConfig() != null && _tableConfig.getIngestionConfig().getBatchIngestionConfig() != null
+            && CollectionUtils.isNotEmpty(
+            _tableConfig.getIngestionConfig().getBatchIngestionConfig().getBatchConfigMaps()),
+        "Must provide ingestionConfig->batchIngestionConfig->batchConfigMaps in tableConfig for table: %s",
+        _tableNameWithType);
+    _batchIngestionConfig = _tableConfig.getIngestionConfig().getBatchIngestionConfig();
+    Preconditions.checkState(_batchIngestionConfig.getBatchConfigMaps().size() == 1,
+        "batchConfigMaps must contain only 1 BatchConfig for table: %s", _tableNameWithType);
+
+    Map<String, String> batchConfigMap = _batchIngestionConfig.getBatchConfigMaps().get(0);
+    String segmentNamePostfixProp = String.format("%s.%s", BatchConfigProperties.SEGMENT_NAME_GENERATOR_PROP_PREFIX,
+        BatchConfigProperties.SEGMENT_NAME_POSTFIX);
+    String segmentSuffix = batchConfigMap.get(segmentNamePostfixProp);
+    segmentSuffix = segmentSuffix == null ? String.valueOf(_indexOfSubtask) : segmentSuffix + "_" + _indexOfSubtask;
+    batchConfigMap.put(segmentNamePostfixProp, segmentSuffix);
+
+    _batchConfig = new BatchConfig(_tableNameWithType, batchConfigMap);
+
+    Preconditions.checkState(StringUtils.isNotBlank(_batchConfig.getOutputDirURI()),
+        "Must provide: %s in batchConfigs for table: %s", BatchConfigProperties.OUTPUT_DIR_URI, _tableNameWithType);
+    _outputDirURI = _batchConfig.getOutputDirURI();
+    Files.createDirectories(Paths.get(_outputDirURI));
+
+    _schema = schema;
+    _fieldsToRead = _schema.getColumnNames();
+    _recordTransformer = CompositeTransformer.getDefaultTransformer(_tableConfig, _schema);
+    _avroSchema = SegmentProcessorAvroUtils.convertPinotSchemaToAvroSchema(_schema);
+    _reusableRecord = new GenericData.Record(_avroSchema);
+
+    // Create tmp dir
+    _stagingDir = new File(FileUtils.getTempDirectory(),
+        String.format("segment_writer_staging_%s_%d_%d", _tableNameWithType, _indexOfSubtask,
+            System.currentTimeMillis()));
+    Preconditions.checkState(_stagingDir.mkdirs(), "Failed to create staging dir: %s", _stagingDir.getAbsolutePath());
+
+    // Create buffer file
+    File bufferDir = new File(_stagingDir, "buffer_dir");
+    Preconditions.checkState(bufferDir.mkdirs(), "Failed to create buffer_dir: %s", bufferDir.getAbsolutePath());
+    _bufferFile = new File(bufferDir, "buffer_file");
+    resetBuffer();
+    LOGGER.info("Initialized {} for Pinot table: {}", this.getClass().getName(), _tableNameWithType);
+  }
+
+  private void registerMetrics(MetricGroup metricGrp) {
+    _processedRecords = metricGrp.counter("records.processed");
+    metricGrp.gauge("record.processing.time.ts", (Gauge<Long>) () -> _lastRecordProcessingTimeMs);
+  }
+
+  private void resetBuffer()
+      throws IOException {
+    FileUtils.deleteQuietly(_bufferFile);
+    _rowCount = 0;
+    _recordWriter = new DataFileWriter<>(new GenericDatumWriter<>(_avroSchema));
+    _recordWriter.create(_avroSchema, _bufferFile);
+  }
+
+  @Override
+  public void collect(GenericRow row)
+      throws IOException {
+    long startTime = System.currentTimeMillis();
+    GenericRow transform = _recordTransformer.transform(row);
+    SegmentProcessorAvroUtils.convertGenericRowToAvroRecord(transform, _reusableRecord, _fieldsToRead);
+    _rowCount++;
+    _recordWriter.append(_reusableRecord);
+    _lastRecordProcessingTimeMs = System.currentTimeMillis() - startTime;
+    _processedRecords.inc();
+  }
+
+  /**
+   * Creates one Pinot segment using the {@link GenericRow}s collected in the AVRO file buffer, at
+   * the outputDirUri as specified in the tableConfig->batchConfigs. Successful invocation of this
+   * method means that the {@link GenericRow}s collected so far, are now available in the Pinot
+   * segment and not available in the buffer anymore.
+   *
+   * <p>Successful completion of segment will return the segment URI, and the URI includes a
+   * sequence id indicating the part number. The sequence id is initialized to 0 and each successful
+   * flush will increment the sequence id by 1. The buffer will be reset and ready to accept further
+   * records via <code>collect()</code> If an exception is thrown, the buffer will not be reset and
+   * so, <code>flush()</code> can be invoked repeatedly in a retry loop. If a successful invocation
+   * is not achieved,<code>close()</code> followed by <code>init </code> will have to be called in
+   * order to reset the buffer and resume record writing.
+   *
+   * @return URI of the generated segment
+   * @throws IOException
+   */
+  @Override
+  public URI flush()
+      throws IOException {
+
+    LOGGER.info("Beginning flush for Pinot table: {} with {} records", _tableNameWithType, _rowCount);
+    _recordWriter.close();
+
+    // Create temp dir for flush
+    File flushDir = new File(_stagingDir, "flush_dir_" + System.currentTimeMillis());
+    Preconditions.checkState(flushDir.mkdirs(), "Failed to create flush dir: %s", flushDir);
+
+    try {
+      // Segment dir
+      File segmentDir = new File(flushDir, "segment_dir");
+
+      // Make BatchIngestionConfig for flush
+      Map<String, String> batchConfigMapOverride = new HashMap<>(_batchConfig.getBatchConfigMap());
+      batchConfigMapOverride.put(BatchConfigProperties.INPUT_DIR_URI, _bufferFile.getAbsolutePath());
+      batchConfigMapOverride.put(BatchConfigProperties.OUTPUT_DIR_URI, segmentDir.getAbsolutePath());
+      batchConfigMapOverride.put(BatchConfigProperties.INPUT_FORMAT, BUFFER_FILE_FORMAT.toString());
+      BatchIngestionConfig batchIngestionConfig = new BatchIngestionConfig(Lists.newArrayList(batchConfigMapOverride),
+          _batchIngestionConfig.getSegmentIngestionType(), _batchIngestionConfig.getSegmentIngestionFrequency());
+
+      // Build segment
+      SegmentGeneratorConfig segmentGeneratorConfig =
+          IngestionUtils.generateSegmentGeneratorConfig(_tableConfig, _schema, batchIngestionConfig);
+      segmentGeneratorConfig.setSequenceId(_seqId);
+      String segmentName = IngestionUtils.buildSegment(segmentGeneratorConfig);
+      LOGGER.info("Successfully built segment: {} of sequence {} for Pinot table: {}", segmentName, _seqId,
+          _tableNameWithType);
+
+      // Tar segment
+      File segmentTarFile =
+          new File(_outputDirURI, String.format("%s_%d%s", segmentName, _indexOfSubtask, Constants.TAR_GZ_FILE_EXT));
+      if (!_batchConfig.isOverwriteOutput() && segmentTarFile.exists()) {
+        segmentTarFile = new File(_outputDirURI,
+            String.format("%s_%d_%d%s", segmentName, _indexOfSubtask, System.currentTimeMillis(),
+                Constants.TAR_GZ_FILE_EXT));
+      }

Review comment:
       if batchConfig.isOverwriteOutput and segmentTarFile.exists, delete the existing file ? Or is the tar creation guaranteed to not get interfered by existence of the file already?

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,266 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a buffer. The {@link GenericRow} are written to
+ * the buffer as AVRO records.
+ */
+@SuppressWarnings("NullAway")
+@NotThreadSafe
+public class FlinkSegmentWriter implements SegmentWriter {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FlinkSegmentWriter.class);
+  private static final FileFormat BUFFER_FILE_FORMAT = FileFormat.AVRO;
+
+  private final int _indexOfSubtask;
+
+  private TableConfig _tableConfig;
+  private String _tableNameWithType;
+  private BatchIngestionConfig _batchIngestionConfig;
+  private BatchConfig _batchConfig;
+  private String _outputDirURI;
+  private Schema _schema;
+  private Set<String> _fieldsToRead;
+  private RecordTransformer _recordTransformer;
+
+  private File _stagingDir;
+  private File _bufferFile;
+  private int _rowCount;
+  /** A sequence ID that increments each time a segment is flushed */
+  private int _seqId;
+
+  private org.apache.avro.Schema _avroSchema;
+  private DataFileWriter<GenericData.Record> _recordWriter;
+  private GenericData.Record _reusableRecord;
+
+  // metrics
+  private transient Counter _processedRecords;
+  private transient volatile long _lastRecordProcessingTimeMs = 0;
+

Review comment:
       nit: s/metricGrp/metricGroup

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,266 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**

Review comment:
       this is the javadoc from FileBasedSegmentWriter. call out here why FlinkSegmentWriter needed to be added, why it is different from FileBased, and any todos to unify the 2. imo, FlinkSegmentWriter should just extend the FileBasedSegmentWriter for the flink metrics related special casing. Everything else should get folded into FileBased impl.

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentUploader.java
##########
@@ -0,0 +1,114 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.spi.auth.AuthContext;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.filesystem.PinotFS;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.uploader.SegmentUploader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Default implementation of {@link SegmentUploader} with support for all push modes The configs for

Review comment:
       javadoc needs to be updated

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,266 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a buffer. The {@link GenericRow} are written to
+ * the buffer as AVRO records.
+ */
+@SuppressWarnings("NullAway")
+@NotThreadSafe
+public class FlinkSegmentWriter implements SegmentWriter {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FlinkSegmentWriter.class);
+  private static final FileFormat BUFFER_FILE_FORMAT = FileFormat.AVRO;
+
+  private final int _indexOfSubtask;
+
+  private TableConfig _tableConfig;
+  private String _tableNameWithType;
+  private BatchIngestionConfig _batchIngestionConfig;
+  private BatchConfig _batchConfig;
+  private String _outputDirURI;
+  private Schema _schema;
+  private Set<String> _fieldsToRead;
+  private RecordTransformer _recordTransformer;
+
+  private File _stagingDir;
+  private File _bufferFile;
+  private int _rowCount;
+  /** A sequence ID that increments each time a segment is flushed */

Review comment:
       what's the main motivation for adding the indexOfSubtask? From Flink side, you want to launch several subtasks, and you want to see that in the segment name/temp dirs for debugging ?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r815380337



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/PinotSinkFunction.java
##########
@@ -0,0 +1,169 @@
+/**
+ * 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.connector.flink.sink;
+
+import java.net.URI;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.pinot.connector.flink.common.RecordConverter;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.ingestion.segment.uploader.SegmentUploader;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The sink function for Pinot.
+ *
+ * <p>This version of the sink function doesn't leverage {@link SegmentWriter} API's ability buffer
+ * data and also share that data with checkpoint state. Instead it uses an internal buffer within
+ * PinotSinkFunction for checkpoint.
+ *
+ * <p>This should change once we introduce FlinkPinotSegmentWriter
+ *
+ * @param <T> type of record supported
+ */
+@SuppressWarnings("NullAway")
+public class PinotSinkFunction<T> extends RichSinkFunction<T> implements CheckpointedFunction {
+
+  public static final long DEFAULT_SEGMENT_FLUSH_MAX_NUM_RECORDS = 500000;
+  public static final int DEFAULT_EXECUTOR_POOL_SIZE = 5;
+  public static final long DEFAULT_EXECUTOR_SHUTDOWN_WAIT_MS = 3000;
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(PinotSinkFunction.class);
+
+  private final long _segmentFlushMaxNumRecords;
+  private final int _executorPoolSize;
+
+  private final RecordConverter<T> _recordConverter;
+
+  private TableConfig _tableConfig;
+  private Schema _schema;
+
+  private transient SegmentWriter _segmentWriter;
+  private transient SegmentUploader _segmentUploader;
+  private transient ExecutorService _executor;
+  private transient long _segmentNumRecord;
+
+  public PinotSinkFunction(RecordConverter<T> recordConverter, TableConfig tableConfig, Schema schema) {
+    this(recordConverter, tableConfig, schema, DEFAULT_SEGMENT_FLUSH_MAX_NUM_RECORDS, DEFAULT_EXECUTOR_POOL_SIZE);
+  }
+
+  public PinotSinkFunction(RecordConverter<T> recordConverter, TableConfig tableConfig, Schema schema,
+      long segmentFlushMaxNumRecords, int executorPoolSize) {
+    _recordConverter = recordConverter;
+    _tableConfig = tableConfig;
+    _schema = schema;
+    _segmentFlushMaxNumRecords = segmentFlushMaxNumRecords;
+    _executorPoolSize = executorPoolSize;
+  }
+
+  @Override
+  public void open(Configuration parameters)
+      throws Exception {
+    int indexOfSubtask = this.getRuntimeContext().getIndexOfThisSubtask();
+    // TODO improve segment uploader to use in-memory buffer then flush to tar file.
+    _segmentWriter = new FlinkSegmentWriter(indexOfSubtask, getRuntimeContext().getMetricGroup());
+    _segmentWriter.init(_tableConfig, _schema);
+    // TODO improve segment uploader to take in-memory tar
+    // TODO launch segment uploader as separate thread for uploading (non-blocking?)
+    _segmentUploader = new FlinkSegmentUploader(indexOfSubtask);
+    _segmentUploader.init(_tableConfig);
+    _segmentNumRecord = 0;
+    _executor = Executors.newFixedThreadPool(_executorPoolSize);
+    LOG.info("Open Pinot Sink with the table {}", _tableConfig.toJsonString());
+  }
+
+  @Override
+  public void close()
+      throws Exception {
+    LOG.info("Closing Pinot Sink");
+    try {
+      if (_segmentNumRecord > 0) {
+        flush();
+      }
+    } catch (Exception e) {
+      LOG.error("Error when closing Pinot sink", e);
+    }
+    _executor.shutdown();
+    try {
+      if (!_executor.awaitTermination(DEFAULT_EXECUTOR_SHUTDOWN_WAIT_MS, TimeUnit.MILLISECONDS)) {
+        _executor.shutdownNow();
+      }
+    } catch (InterruptedException e) {
+      _executor.shutdownNow();
+    }
+    _segmentWriter.close();
+  }
+
+  @Override
+  public void invoke(T value, Context context)
+      throws Exception {
+    _segmentWriter.collect(_recordConverter.convertToRow(value));
+    _segmentNumRecord++;
+    if (_segmentNumRecord > _segmentFlushMaxNumRecords) {
+      flush();
+    }
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext functionSnapshotContext)
+      throws Exception {
+    // TODO: not supported yet
+    LOG.error("snapshotState is invoked in Pinot sink");
+    // clear and flush.
+    flush();
+    // snapshot state:
+    // 1. should only work on the boundary of segment uploader.
+    // 2. segmentwriter state should be preserved.
+    // 3.
+    // ...
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext functionInitializationContext)
+      throws Exception {
+    // no initialization needed
+    // ...
+  }
+
+  private void flush()

Review comment:
       sure




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (4a01a0f) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `6.97%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     71.06%   64.08%   -6.98%     
   + Complexity     4320     4241      -79     
   ============================================
     Files          1626     1586      -40     
     Lines         85036    83399    -1637     
     Branches      12795    12641     -154     
   ============================================
   - Hits          60428    53450    -6978     
   - Misses        20456    26100    +5644     
   + Partials       4152     3849     -303     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `66.98% <ø> (-0.37%)` | :arrow_down: |
   | unittests2 | `14.15% <ø> (+0.07%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ache/pinot/server/access/AccessControlFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VydmVyL2FjY2Vzcy9BY2Nlc3NDb250cm9sRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/SegmentReloadMessage.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvU2VnbWVudFJlbG9hZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/core/data/manager/realtime/TimerService.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvVGltZXJTZXJ2aWNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...che/pinot/controller/util/TableMetadataReader.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci91dGlsL1RhYmxlTWV0YWRhdGFSZWFkZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/minion/exception/TaskCancelledException.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vZXhjZXB0aW9uL1Rhc2tDYW5jZWxsZWRFeGNlcHRpb24uamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...not/common/exception/HttpErrorStatusException.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL0h0dHBFcnJvclN0YXR1c0V4Y2VwdGlvbi5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [421 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...4a01a0f](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r815380762



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/HttpClient.java
##########
@@ -0,0 +1,130 @@
+/**

Review comment:
       Not really. I think this is an integral part to show how the connector is used as in `QuickStart`, without it I don't think the connector is complete. It doesn't make sense to expect developers to manually create the Schema as well as TableConfig, not to mention several configurations decoration needed in the `PinotConnectionUtils `.
   
   Xiang's point is that in later PR we can refactor this part of code with a reusable client lib. However, from the connector's user perspective, it doesn't change the way of using the connector. 
   
   Makes sense?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] walterddr commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
walterddr commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r815327278



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/common/PinotMapRecordConverter.java
##########
@@ -0,0 +1,41 @@
+/**
+ * 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.connector.flink.common;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+public class PinotMapRecordConverter implements RecordConverter<Map> {

Review comment:
       ```suggestion
   public class PinotMapRecordConverter implements RecordConverter<Map<String, Object>> {
   ```
   to avoid unchecked casts.

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/PinotSinkFunction.java
##########
@@ -0,0 +1,169 @@
+/**
+ * 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.connector.flink.sink;
+
+import java.net.URI;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.pinot.connector.flink.common.RecordConverter;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.ingestion.segment.uploader.SegmentUploader;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The sink function for Pinot.
+ *
+ * <p>This version of the sink function doesn't leverage {@link SegmentWriter} API's ability buffer
+ * data and also share that data with checkpoint state. Instead it uses an internal buffer within
+ * PinotSinkFunction for checkpoint.
+ *
+ * <p>This should change once we introduce FlinkPinotSegmentWriter

Review comment:
       this javadoc is outdated.

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentUploader.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.spi.auth.AuthContext;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.filesystem.PinotFS;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.uploader.SegmentUploader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Default implementation of {@link SegmentUploader} with support for all push modes The configs for
+ * push are fetched from batchConfigMaps of tableConfig
+ */
+@SuppressWarnings("NullAway")
+public class FlinkSegmentUploader implements SegmentUploader {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FlinkSegmentUploader.class);
+  private final int _indexOfSubtask;
+
+  private String _tableNameWithType;
+  private BatchConfig _batchConfig;
+  private BatchIngestionConfig _batchIngestionConfig;
+
+  public FlinkSegmentUploader(int indexOfSubtask) {
+    _indexOfSubtask = indexOfSubtask;

Review comment:
       indexOfSubtask is not used in uploaded. I think it only matters in segment writer. 

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,267 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a buffer. The {@link GenericRow} are written to
+ * the buffer as AVRO records.
+ */
+@SuppressWarnings("NullAway")
+@NotThreadSafe
+public class FlinkSegmentWriter implements SegmentWriter {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FlinkSegmentWriter.class);
+  private static final FileFormat BUFFER_FILE_FORMAT = FileFormat.AVRO;
+
+  private final int _indexOfSubtask;
+
+  private TableConfig _tableConfig;
+  private String _tableNameWithType;
+  private BatchIngestionConfig _batchIngestionConfig;
+  private BatchConfig _batchConfig;
+  private String _outputDirURI;
+  private Schema _schema;
+  private Set<String> _fieldsToRead;
+  private RecordTransformer _recordTransformer;
+
+  private File _stagingDir;
+  private File _bufferFile;
+  private int _rowCount;
+  /** A sequence ID that increments each time a segment is flushed */
+  private int _seqId;
+
+  private org.apache.avro.Schema _avroSchema;
+  private DataFileWriter<GenericData.Record> _recordWriter;
+  private GenericData.Record _reusableRecord;
+
+  // metrics
+  private transient Counter _processedRecords;
+  private transient volatile long _lastRecordProcessingTimeMs = 0;
+
+  public FlinkSegmentWriter(int indexOfSubtask, MetricGroup metricGrp) {
+    _indexOfSubtask = indexOfSubtask;
+    registerMetrics(metricGrp);
+  }
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema)
+      throws Exception {
+    init(tableConfig, schema, null);
+  }
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema, Map<String, String> batchConfigOverride)
+      throws Exception {
+    _rowCount = 0;
+    _seqId = 0;
+    _tableConfig = tableConfig;
+    _tableNameWithType = _tableConfig.getTableName();
+    Preconditions.checkState(
+        _tableConfig.getIngestionConfig() != null && _tableConfig.getIngestionConfig().getBatchIngestionConfig() != null
+            && CollectionUtils.isNotEmpty(
+            _tableConfig.getIngestionConfig().getBatchIngestionConfig().getBatchConfigMaps()),
+        "Must provide ingestionConfig->batchIngestionConfig->batchConfigMaps in tableConfig for table: %s",
+        _tableNameWithType);
+    _batchIngestionConfig = _tableConfig.getIngestionConfig().getBatchIngestionConfig();
+    Preconditions.checkState(_batchIngestionConfig.getBatchConfigMaps().size() == 1,
+        "batchConfigMaps must contain only 1 BatchConfig for table: %s", _tableNameWithType);
+
+    Map<String, String> batchConfigMap = _batchIngestionConfig.getBatchConfigMaps().get(0);
+    String segmentNamePostfixProp = String.format("%s.%s", BatchConfigProperties.SEGMENT_NAME_GENERATOR_PROP_PREFIX,
+        BatchConfigProperties.SEGMENT_NAME_POSTFIX);
+    String segmentSuffix = batchConfigMap.get(segmentNamePostfixProp);
+    segmentSuffix = segmentSuffix == null ? String.valueOf(_indexOfSubtask) : segmentSuffix + "_" + _indexOfSubtask;
+    batchConfigMap.put(segmentNamePostfixProp, segmentSuffix);
+
+    _batchConfig = new BatchConfig(_tableNameWithType, batchConfigMap);
+
+    Preconditions.checkState(StringUtils.isNotBlank(_batchConfig.getOutputDirURI()),
+        "Must provide: %s in batchConfigs for table: %s", BatchConfigProperties.OUTPUT_DIR_URI, _tableNameWithType);
+    _outputDirURI = _batchConfig.getOutputDirURI();
+    Files.createDirectories(Paths.get(_outputDirURI));
+
+    _schema = schema;
+    _fieldsToRead = _schema.getColumnNames();
+    _recordTransformer = CompositeTransformer.getDefaultTransformer(_tableConfig, _schema);
+    _avroSchema = SegmentProcessorAvroUtils.convertPinotSchemaToAvroSchema(_schema);
+    _reusableRecord = new GenericData.Record(_avroSchema);
+
+    // Create tmp dir
+    // TODO staging Dir also need to be subtask separated otherwise there will be write conflict.

Review comment:
       clean up TODO

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,267 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a buffer. The {@link GenericRow} are written to
+ * the buffer as AVRO records.
+ */
+@SuppressWarnings("NullAway")
+@NotThreadSafe
+public class FlinkSegmentWriter implements SegmentWriter {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FlinkSegmentWriter.class);
+  private static final FileFormat BUFFER_FILE_FORMAT = FileFormat.AVRO;
+
+  private final int _indexOfSubtask;
+
+  private TableConfig _tableConfig;
+  private String _tableNameWithType;
+  private BatchIngestionConfig _batchIngestionConfig;
+  private BatchConfig _batchConfig;
+  private String _outputDirURI;
+  private Schema _schema;
+  private Set<String> _fieldsToRead;
+  private RecordTransformer _recordTransformer;
+
+  private File _stagingDir;
+  private File _bufferFile;
+  private int _rowCount;
+  /** A sequence ID that increments each time a segment is flushed */
+  private int _seqId;
+
+  private org.apache.avro.Schema _avroSchema;
+  private DataFileWriter<GenericData.Record> _recordWriter;
+  private GenericData.Record _reusableRecord;
+
+  // metrics
+  private transient Counter _processedRecords;
+  private transient volatile long _lastRecordProcessingTimeMs = 0;
+
+  public FlinkSegmentWriter(int indexOfSubtask, MetricGroup metricGrp) {

Review comment:
       nit: Refactor common code between FlinkSegmentWriter and FileBaseSegmentWriter but can be done in separate PR

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/PinotSinkFunction.java
##########
@@ -0,0 +1,169 @@
+/**
+ * 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.connector.flink.sink;
+
+import java.net.URI;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.pinot.connector.flink.common.RecordConverter;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.ingestion.segment.uploader.SegmentUploader;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The sink function for Pinot.
+ *
+ * <p>This version of the sink function doesn't leverage {@link SegmentWriter} API's ability buffer
+ * data and also share that data with checkpoint state. Instead it uses an internal buffer within
+ * PinotSinkFunction for checkpoint.
+ *
+ * <p>This should change once we introduce FlinkPinotSegmentWriter
+ *
+ * @param <T> type of record supported
+ */
+@SuppressWarnings("NullAway")
+public class PinotSinkFunction<T> extends RichSinkFunction<T> implements CheckpointedFunction {
+
+  public static final long DEFAULT_SEGMENT_FLUSH_MAX_NUM_RECORDS = 500000;
+  public static final int DEFAULT_EXECUTOR_POOL_SIZE = 5;
+  public static final long DEFAULT_EXECUTOR_SHUTDOWN_WAIT_MS = 3000;
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(PinotSinkFunction.class);
+
+  private final long _segmentFlushMaxNumRecords;
+  private final int _executorPoolSize;
+
+  private final RecordConverter<T> _recordConverter;
+
+  private TableConfig _tableConfig;
+  private Schema _schema;
+
+  private transient SegmentWriter _segmentWriter;
+  private transient SegmentUploader _segmentUploader;
+  private transient ExecutorService _executor;
+  private transient long _segmentNumRecord;
+
+  public PinotSinkFunction(RecordConverter<T> recordConverter, TableConfig tableConfig, Schema schema) {
+    this(recordConverter, tableConfig, schema, DEFAULT_SEGMENT_FLUSH_MAX_NUM_RECORDS, DEFAULT_EXECUTOR_POOL_SIZE);
+  }
+
+  public PinotSinkFunction(RecordConverter<T> recordConverter, TableConfig tableConfig, Schema schema,
+      long segmentFlushMaxNumRecords, int executorPoolSize) {
+    _recordConverter = recordConverter;
+    _tableConfig = tableConfig;
+    _schema = schema;
+    _segmentFlushMaxNumRecords = segmentFlushMaxNumRecords;
+    _executorPoolSize = executorPoolSize;
+  }
+
+  @Override
+  public void open(Configuration parameters)
+      throws Exception {
+    int indexOfSubtask = this.getRuntimeContext().getIndexOfThisSubtask();
+    // TODO improve segment uploader to use in-memory buffer then flush to tar file.
+    _segmentWriter = new FlinkSegmentWriter(indexOfSubtask, getRuntimeContext().getMetricGroup());
+    _segmentWriter.init(_tableConfig, _schema);
+    // TODO improve segment uploader to take in-memory tar
+    // TODO launch segment uploader as separate thread for uploading (non-blocking?)
+    _segmentUploader = new FlinkSegmentUploader(indexOfSubtask);
+    _segmentUploader.init(_tableConfig);
+    _segmentNumRecord = 0;
+    _executor = Executors.newFixedThreadPool(_executorPoolSize);
+    LOG.info("Open Pinot Sink with the table {}", _tableConfig.toJsonString());
+  }
+
+  @Override
+  public void close()
+      throws Exception {
+    LOG.info("Closing Pinot Sink");
+    try {
+      if (_segmentNumRecord > 0) {
+        flush();
+      }
+    } catch (Exception e) {
+      LOG.error("Error when closing Pinot sink", e);
+    }
+    _executor.shutdown();
+    try {
+      if (!_executor.awaitTermination(DEFAULT_EXECUTOR_SHUTDOWN_WAIT_MS, TimeUnit.MILLISECONDS)) {
+        _executor.shutdownNow();
+      }
+    } catch (InterruptedException e) {
+      _executor.shutdownNow();
+    }
+    _segmentWriter.close();
+  }
+
+  @Override
+  public void invoke(T value, Context context)
+      throws Exception {
+    _segmentWriter.collect(_recordConverter.convertToRow(value));
+    _segmentNumRecord++;
+    if (_segmentNumRecord > _segmentFlushMaxNumRecords) {
+      flush();
+    }
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext functionSnapshotContext)
+      throws Exception {
+    // TODO: not supported yet
+    LOG.error("snapshotState is invoked in Pinot sink");
+    // clear and flush.
+    flush();
+    // snapshot state:
+    // 1. should only work on the boundary of segment uploader.
+    // 2. segmentwriter state should be preserved.
+    // 3.
+    // ...
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext functionInitializationContext)
+      throws Exception {
+    // no initialization needed
+    // ...
+  }
+
+  private void flush()

Review comment:
       potential race condition when snapshot and invoke both calls flush?

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/HttpClient.java
##########
@@ -0,0 +1,130 @@
+/**

Review comment:
       I might've been wrong but I dont see the entire package of `org.apache.pinot.connector.flink.http` used in anywhere outside of this package.
   
   is this module relevant to the flink connector PR at all? or this is simply used for integration/e2e test as an easy util? 
   
   i've deleted this entire module and tests runs just fine, so if not relevant I suggest we delete this module 

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/PinotSinkFunction.java
##########
@@ -0,0 +1,169 @@
+/**
+ * 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.connector.flink.sink;
+
+import java.net.URI;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.pinot.connector.flink.common.RecordConverter;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.ingestion.segment.uploader.SegmentUploader;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The sink function for Pinot.
+ *
+ * <p>This version of the sink function doesn't leverage {@link SegmentWriter} API's ability buffer
+ * data and also share that data with checkpoint state. Instead it uses an internal buffer within
+ * PinotSinkFunction for checkpoint.
+ *
+ * <p>This should change once we introduce FlinkPinotSegmentWriter
+ *
+ * @param <T> type of record supported
+ */
+@SuppressWarnings("NullAway")
+public class PinotSinkFunction<T> extends RichSinkFunction<T> implements CheckpointedFunction {
+
+  public static final long DEFAULT_SEGMENT_FLUSH_MAX_NUM_RECORDS = 500000;
+  public static final int DEFAULT_EXECUTOR_POOL_SIZE = 5;
+  public static final long DEFAULT_EXECUTOR_SHUTDOWN_WAIT_MS = 3000;
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(PinotSinkFunction.class);
+
+  private final long _segmentFlushMaxNumRecords;
+  private final int _executorPoolSize;
+
+  private final RecordConverter<T> _recordConverter;
+
+  private TableConfig _tableConfig;
+  private Schema _schema;
+
+  private transient SegmentWriter _segmentWriter;
+  private transient SegmentUploader _segmentUploader;
+  private transient ExecutorService _executor;
+  private transient long _segmentNumRecord;
+
+  public PinotSinkFunction(RecordConverter<T> recordConverter, TableConfig tableConfig, Schema schema) {
+    this(recordConverter, tableConfig, schema, DEFAULT_SEGMENT_FLUSH_MAX_NUM_RECORDS, DEFAULT_EXECUTOR_POOL_SIZE);
+  }
+
+  public PinotSinkFunction(RecordConverter<T> recordConverter, TableConfig tableConfig, Schema schema,
+      long segmentFlushMaxNumRecords, int executorPoolSize) {
+    _recordConverter = recordConverter;
+    _tableConfig = tableConfig;
+    _schema = schema;
+    _segmentFlushMaxNumRecords = segmentFlushMaxNumRecords;
+    _executorPoolSize = executorPoolSize;
+  }
+
+  @Override
+  public void open(Configuration parameters)
+      throws Exception {
+    int indexOfSubtask = this.getRuntimeContext().getIndexOfThisSubtask();
+    // TODO improve segment uploader to use in-memory buffer then flush to tar file.
+    _segmentWriter = new FlinkSegmentWriter(indexOfSubtask, getRuntimeContext().getMetricGroup());
+    _segmentWriter.init(_tableConfig, _schema);
+    // TODO improve segment uploader to take in-memory tar
+    // TODO launch segment uploader as separate thread for uploading (non-blocking?)

Review comment:
       clean up TODOs. 

##########
File path: pinot-connectors/pinot-flink-connector/pom.xml
##########
@@ -0,0 +1,173 @@
+<?xml version="1.0"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.pinot</groupId>
+    <artifactId>pinot-connectors</artifactId>
+    <version>0.10.0-SNAPSHOT</version>
+    <relativePath>..</relativePath>
+  </parent>
+  <artifactId>flink-pinot-sink</artifactId>
+  <name>Pinot Flink Connector</name>
+  <url>https://pinot.apache.org/</url>
+  <properties>
+    <pinot.root>${basedir}/../..</pinot.root>
+    <scala.version>2.12</scala.version>
+    <flink.version>1.12.0</flink.version>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.asynchttpclient</groupId>
+      <artifactId>async-http-client</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-unix-common</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.inject</groupId>
+      <artifactId>jersey-hk2</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.media</groupId>
+      <artifactId>jersey-media-json-jackson</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-streaming-java_${scala.version}</artifactId>
+      <version>${flink.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.pinot</groupId>
+      <artifactId>pinot-segment-writer-file-based</artifactId>
+      <version>${project.parent.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.pinot</groupId>
+      <artifactId>pinot-segment-uploader-default</artifactId>
+      <version>${project.parent.version}</version>
+    </dependency>
+
+    <!-- Test Dependencies -->
+    <dependency>
+      <groupId>org.testng</groupId>
+      <artifactId>testng</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-clients_${scala.version}</artifactId>
+      <version>${flink.version}</version>
+    </dependency>

Review comment:
       add `<scope>test</scope>` to these 2 dependencies.

##########
File path: pinot-connectors/pinot-flink-connector/pom.xml
##########
@@ -0,0 +1,173 @@
+<?xml version="1.0"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.pinot</groupId>
+    <artifactId>pinot-connectors</artifactId>
+    <version>0.10.0-SNAPSHOT</version>
+    <relativePath>..</relativePath>
+  </parent>
+  <artifactId>flink-pinot-sink</artifactId>

Review comment:
       ```suggestion
     <artifactId>pinot-flink-connector</artifactId>
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d68d991) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `1.41%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     71.06%   69.64%   -1.42%     
   + Complexity     4320     4242      -78     
   ============================================
     Files          1626     1631       +5     
     Lines         85036    85281     +245     
     Branches      12795    12845      +50     
   ============================================
   - Hits          60428    59393    -1035     
   - Misses        20456    21750    +1294     
   + Partials       4152     4138      -14     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `28.68% <ø> (-0.18%)` | :arrow_down: |
   | integration2 | `?` | |
   | unittests1 | `66.98% <ø> (-0.37%)` | :arrow_down: |
   | unittests2 | `14.12% <ø> (+0.04%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...t/core/plan/StreamingInstanceResponsePlanNode.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9wbGFuL1N0cmVhbWluZ0luc3RhbmNlUmVzcG9uc2VQbGFuTm9kZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ore/operator/streaming/StreamingResponseUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9zdHJlYW1pbmcvU3RyZWFtaW5nUmVzcG9uc2VVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ager/realtime/PeerSchemeSplitSegmentCommitter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUGVlclNjaGVtZVNwbGl0U2VnbWVudENvbW1pdHRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/core/plan/StreamingSelectionPlanNode.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9wbGFuL1N0cmVhbWluZ1NlbGVjdGlvblBsYW5Ob2RlLmphdmE=) | `0.00% <0.00%> (-88.89%)` | :arrow_down: |
   | [...ator/streaming/StreamingSelectionOnlyOperator.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9zdHJlYW1pbmcvU3RyZWFtaW5nU2VsZWN0aW9uT25seU9wZXJhdG9yLmphdmE=) | `0.00% <0.00%> (-87.81%)` | :arrow_down: |
   | [...re/query/reduce/SelectionOnlyStreamingReducer.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvU2VsZWN0aW9uT25seVN0cmVhbWluZ1JlZHVjZXIuamF2YQ==) | `0.00% <0.00%> (-85.72%)` | :arrow_down: |
   | [...pache/pinot/common/utils/grpc/GrpcQueryClient.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvZ3JwYy9HcnBjUXVlcnlDbGllbnQuamF2YQ==) | `0.00% <0.00%> (-80.86%)` | :arrow_down: |
   | [...data/manager/realtime/DefaultSegmentCommitter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvRGVmYXVsdFNlZ21lbnRDb21taXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-80.00%)` | :arrow_down: |
   | [...ller/api/access/BasicAuthAccessControlFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvYWNjZXNzL0Jhc2ljQXV0aEFjY2Vzc0NvbnRyb2xGYWN0b3J5LmphdmE=) | `0.00% <0.00%> (-80.00%)` | :arrow_down: |
   | [...roker/requesthandler/GrpcBrokerRequestHandler.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcmVxdWVzdGhhbmRsZXIvR3JwY0Jyb2tlclJlcXVlc3RIYW5kbGVyLmphdmE=) | `0.00% <0.00%> (-78.58%)` | :arrow_down: |
   | ... and [151 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...d68d991](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 merged pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 merged pull request #8233:
URL: https://github.com/apache/pinot/pull/8233


   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r813101401



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/HttpClient.java
##########
@@ -0,0 +1,130 @@
+/**
+ * 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.connector.flink.http;
+
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import org.glassfish.jersey.client.JerseyClientBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/** This is copied from {@link HttpClient} in RTA-UMS project. */
+public abstract class HttpClient {

Review comment:
       yeah, it's just some helper methods that can be reused by subclasses. For example, if we want to add other clients to broker/server etc




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r813361808



##########
File path: pinot-connectors/pinot-flink-connector/src/test/java/org/apache/pinot/connector/flink/http/PinotControllerClientTest.java
##########
@@ -0,0 +1,168 @@
+/**
+ * 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.connector.flink.http;
+
+import com.google.common.collect.Lists;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.Invocation;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedHashMap;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import org.apache.commons.io.IOUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.testng.annotations.Test;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.assertEquals;
+
+
+public class PinotControllerClientTest {
+
+  private static final String TABLE_NAME = "demand";
+
+  @Test
+  public void testGetPinotControllerInstancesFromController()
+      throws Exception {
+    Client client = mock(Client.class);
+    WebTarget target = mock(WebTarget.class);
+    Invocation.Builder builder = mock(Invocation.Builder.class);
+    Response response = mock(Response.class);
+    MultivaluedMap<String, Object> headers = getRequestHeadersToPinot();
+    String expectedFullURL = "http://localhost:9000/instances";
+    final Map<String, Object> resEntity =
+        JsonUtils.stringToObject(fixture("fixtures/pinotControllerInstances.json"), Map.class);
+
+    when(client.target(expectedFullURL)).thenReturn(target);
+    when(target.request()).thenReturn(builder);
+    when(builder.headers(headers)).thenReturn(builder);
+    when(builder.get()).thenReturn(response);
+    when(response.getStatus()).thenReturn(200);
+    when(response.getEntity()).thenReturn(resEntity);
+    when(response.readEntity(Map.class)).thenReturn(resEntity);
+
+    PinotControllerClient controllerClient = new PinotControllerClient();
+    controllerClient.setHttpClient(client);
+    List<String> instances = controllerClient.getControllerInstances(new MultivaluedHashMap<>());
+    assertEquals(Lists.newArrayList("pinot-prod02:5983"), instances);
+
+    verify(client, times(1)).target(expectedFullURL);
+    verify(target, times(1)).request();
+    verify(builder, times(1)).headers(headers);
+    verify(builder, times(1)).get();
+  }
+
+  @Test
+  public void testGetPinotSchemaStrFromController()
+      throws Exception {
+    Client client = mock(Client.class);
+    WebTarget target = mock(WebTarget.class);
+    Invocation.Builder builder = mock(Invocation.Builder.class);
+    Response response = mock(Response.class);
+    MultivaluedMap<String, Object> headers = getRequestHeadersToPinot();
+    String expectedFullURL = "http://localhost:9000/schemas/demand";
+    final Map<String, Object> resEntity =
+        JsonUtils.stringToObject(fixture("fixtures/pinotTableSchema.json"), Map.class);
+
+    when(client.target(expectedFullURL)).thenReturn(target);
+    when(target.request()).thenReturn(builder);
+    when(builder.headers(headers)).thenReturn(builder);
+    when(builder.get()).thenReturn(response);
+    when(response.getStatus()).thenReturn(200);
+    when(response.getEntity()).thenReturn(resEntity);
+    when(response.readEntity(Map.class)).thenReturn(resEntity);
+
+    PinotControllerClient controllerClient = new PinotControllerClient();
+    controllerClient.setHttpClient(client);
+    String schemaStrFromController =
+        controllerClient.getSchemaStrFromController(TABLE_NAME, new MultivaluedHashMap<>());
+    assertEquals(
+        JsonUtils.objectToString(JsonUtils.stringToObject(fixture("fixtures/pinotTableSchema.json"), Map.class)),
+        schemaStrFromController);
+
+    verify(client, times(1)).target(expectedFullURL);
+    verify(target, times(1)).request();
+    verify(builder, times(1)).headers(headers);
+    verify(builder, times(1)).get();
+  }
+
+  @Test
+  public void testGetPinotConfigStrFromController()
+      throws Exception {
+    Client client = mock(Client.class);
+    WebTarget target = mock(WebTarget.class);
+    Invocation.Builder builder = mock(Invocation.Builder.class);
+    Response response = mock(Response.class);
+    MultivaluedMap<String, Object> headers = getRequestHeadersToPinot();
+    String expectedFullURL = "http://localhost:9000/tables/demand?type=realtime";
+    final Map<String, Object> resEntity = new HashMap<>() {

Review comment:
       btw, this usage `HashMap<>` is all over the codebase




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2a55ab2) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `6.74%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     71.06%   64.31%   -6.75%     
   + Complexity     4320     4319       -1     
   ============================================
     Files          1626     1581      -45     
     Lines         85036    83194    -1842     
     Branches      12795    12598     -197     
   ============================================
   - Hits          60428    53510    -6918     
   - Misses        20456    25848    +5392     
   + Partials       4152     3836     -316     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `67.35% <ø> (+<0.01%)` | :arrow_up: |
   | unittests2 | `14.12% <ø> (+0.04%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ache/pinot/server/access/AccessControlFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VydmVyL2FjY2Vzcy9BY2Nlc3NDb250cm9sRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/SegmentReloadMessage.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvU2VnbWVudFJlbG9hZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/core/data/manager/realtime/TimerService.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvVGltZXJTZXJ2aWNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...che/pinot/controller/util/TableMetadataReader.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci91dGlsL1RhYmxlTWV0YWRhdGFSZWFkZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/minion/exception/TaskCancelledException.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vZXhjZXB0aW9uL1Rhc2tDYW5jZWxsZWRFeGNlcHRpb24uamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...not/common/exception/HttpErrorStatusException.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL0h0dHBFcnJvclN0YXR1c0V4Y2VwdGlvbi5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [374 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...2a55ab2](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] xiangfu0 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r812487512



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/PinotControllerClient.java
##########
@@ -0,0 +1,147 @@
+/**
+ * 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.connector.flink.http;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonValue;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/** This file is partially copied from RTAUMS gateway package */
+public class PinotControllerClient extends HttpClient {

Review comment:
       I feel this is a very helpful Client. 
   
   Just drop my random thought her:
   Shall we rename this to PinotAdminClient and move it to the `pinot-java-client` module then import it back?
   
   cc: @KKcorps @Jackie-Jiang 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e0cd7b2) into [master](https://codecov.io/gh/apache/pinot/commit/46ed731c4e60c308c9559e46349a984b0ce05ce6?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (46ed731) will **decrease** coverage by `1.32%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     70.83%   69.51%   -1.33%     
     Complexity     4245     4245              
   ============================================
     Files          1631     1631              
     Lines         85462    85490      +28     
     Branches      12877    12878       +1     
   ============================================
   - Hits          60539    59427    -1112     
   - Misses        20746    21923    +1177     
   + Partials       4177     4140      -37     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `27.57% <ø> (-0.04%)` | :arrow_down: |
   | unittests1 | `66.97% <ø> (+0.01%)` | :arrow_up: |
   | unittests2 | `14.17% <ø> (+0.01%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...pinot/minion/exception/TaskCancelledException.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vZXhjZXB0aW9uL1Rhc2tDYW5jZWxsZWRFeGNlcHRpb24uamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...nverttorawindex/ConvertToRawIndexTaskExecutor.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1taW5pb24tdGFza3MvcGlub3QtbWluaW9uLWJ1aWx0aW4tdGFza3Mvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3BsdWdpbi9taW5pb24vdGFza3MvY29udmVydHRvcmF3aW5kZXgvQ29udmVydFRvUmF3SW5kZXhUYXNrRXhlY3V0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...e/pinot/common/minion/MergeRollupTaskMetadata.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWluaW9uL01lcmdlUm9sbHVwVGFza01ldGFkYXRhLmphdmE=) | `0.00% <0.00%> (-94.74%)` | :arrow_down: |
   | [...plugin/segmentuploader/SegmentUploaderDefault.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1zZWdtZW50LXVwbG9hZGVyL3Bpbm90LXNlZ21lbnQtdXBsb2FkZXItZGVmYXVsdC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcGx1Z2luL3NlZ21lbnR1cGxvYWRlci9TZWdtZW50VXBsb2FkZXJEZWZhdWx0LmphdmE=) | `0.00% <0.00%> (-87.10%)` | :arrow_down: |
   | [.../transform/function/MapValueTransformFunction.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vTWFwVmFsdWVUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `0.00% <0.00%> (-85.30%)` | :arrow_down: |
   | [...ot/common/messages/RoutingTableRebuildMessage.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvUm91dGluZ1RhYmxlUmVidWlsZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-81.82%)` | :arrow_down: |
   | [...data/manager/realtime/DefaultSegmentCommitter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvRGVmYXVsdFNlZ21lbnRDb21taXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-80.00%)` | :arrow_down: |
   | [...verttorawindex/ConvertToRawIndexTaskGenerator.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1taW5pb24tdGFza3MvcGlub3QtbWluaW9uLWJ1aWx0aW4tdGFza3Mvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3BsdWdpbi9taW5pb24vdGFza3MvY29udmVydHRvcmF3aW5kZXgvQ29udmVydFRvUmF3SW5kZXhUYXNrR2VuZXJhdG9yLmphdmE=) | `5.45% <0.00%> (-80.00%)` | :arrow_down: |
   | [...ache/pinot/common/lineage/SegmentLineageUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbGluZWFnZS9TZWdtZW50TGluZWFnZVV0aWxzLmphdmE=) | `22.22% <0.00%> (-77.78%)` | :arrow_down: |
   | [...ore/startree/executor/StarTreeGroupByExecutor.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9zdGFydHJlZS9leGVjdXRvci9TdGFyVHJlZUdyb3VwQnlFeGVjdXRvci5qYXZh) | `0.00% <0.00%> (-77.78%)` | :arrow_down: |
   | ... and [107 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [46ed731...e0cd7b2](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] walterddr commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
walterddr commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r815377423



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/PinotSinkFunction.java
##########
@@ -0,0 +1,169 @@
+/**
+ * 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.connector.flink.sink;
+
+import java.net.URI;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.pinot.connector.flink.common.RecordConverter;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.ingestion.segment.uploader.SegmentUploader;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The sink function for Pinot.
+ *
+ * <p>This version of the sink function doesn't leverage {@link SegmentWriter} API's ability buffer
+ * data and also share that data with checkpoint state. Instead it uses an internal buffer within
+ * PinotSinkFunction for checkpoint.
+ *
+ * <p>This should change once we introduce FlinkPinotSegmentWriter
+ *
+ * @param <T> type of record supported
+ */
+@SuppressWarnings("NullAway")
+public class PinotSinkFunction<T> extends RichSinkFunction<T> implements CheckpointedFunction {
+
+  public static final long DEFAULT_SEGMENT_FLUSH_MAX_NUM_RECORDS = 500000;
+  public static final int DEFAULT_EXECUTOR_POOL_SIZE = 5;
+  public static final long DEFAULT_EXECUTOR_SHUTDOWN_WAIT_MS = 3000;
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(PinotSinkFunction.class);
+
+  private final long _segmentFlushMaxNumRecords;
+  private final int _executorPoolSize;
+
+  private final RecordConverter<T> _recordConverter;
+
+  private TableConfig _tableConfig;
+  private Schema _schema;
+
+  private transient SegmentWriter _segmentWriter;
+  private transient SegmentUploader _segmentUploader;
+  private transient ExecutorService _executor;
+  private transient long _segmentNumRecord;
+
+  public PinotSinkFunction(RecordConverter<T> recordConverter, TableConfig tableConfig, Schema schema) {
+    this(recordConverter, tableConfig, schema, DEFAULT_SEGMENT_FLUSH_MAX_NUM_RECORDS, DEFAULT_EXECUTOR_POOL_SIZE);
+  }
+
+  public PinotSinkFunction(RecordConverter<T> recordConverter, TableConfig tableConfig, Schema schema,
+      long segmentFlushMaxNumRecords, int executorPoolSize) {
+    _recordConverter = recordConverter;
+    _tableConfig = tableConfig;
+    _schema = schema;
+    _segmentFlushMaxNumRecords = segmentFlushMaxNumRecords;
+    _executorPoolSize = executorPoolSize;
+  }
+
+  @Override
+  public void open(Configuration parameters)
+      throws Exception {
+    int indexOfSubtask = this.getRuntimeContext().getIndexOfThisSubtask();
+    // TODO improve segment uploader to use in-memory buffer then flush to tar file.
+    _segmentWriter = new FlinkSegmentWriter(indexOfSubtask, getRuntimeContext().getMetricGroup());
+    _segmentWriter.init(_tableConfig, _schema);
+    // TODO improve segment uploader to take in-memory tar
+    // TODO launch segment uploader as separate thread for uploading (non-blocking?)
+    _segmentUploader = new FlinkSegmentUploader(indexOfSubtask);
+    _segmentUploader.init(_tableConfig);
+    _segmentNumRecord = 0;
+    _executor = Executors.newFixedThreadPool(_executorPoolSize);
+    LOG.info("Open Pinot Sink with the table {}", _tableConfig.toJsonString());
+  }
+
+  @Override
+  public void close()
+      throws Exception {
+    LOG.info("Closing Pinot Sink");
+    try {
+      if (_segmentNumRecord > 0) {
+        flush();
+      }
+    } catch (Exception e) {
+      LOG.error("Error when closing Pinot sink", e);
+    }
+    _executor.shutdown();
+    try {
+      if (!_executor.awaitTermination(DEFAULT_EXECUTOR_SHUTDOWN_WAIT_MS, TimeUnit.MILLISECONDS)) {
+        _executor.shutdownNow();
+      }
+    } catch (InterruptedException e) {
+      _executor.shutdownNow();
+    }
+    _segmentWriter.close();
+  }
+
+  @Override
+  public void invoke(T value, Context context)
+      throws Exception {
+    _segmentWriter.collect(_recordConverter.convertToRow(value));
+    _segmentNumRecord++;
+    if (_segmentNumRecord > _segmentFlushMaxNumRecords) {
+      flush();
+    }
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext functionSnapshotContext)
+      throws Exception {
+    // TODO: not supported yet
+    LOG.error("snapshotState is invoked in Pinot sink");
+    // clear and flush.
+    flush();
+    // snapshot state:
+    // 1. should only work on the boundary of segment uploader.
+    // 2. segmentwriter state should be preserved.
+    // 3.
+    // ...
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext functionInitializationContext)
+      throws Exception {
+    // no initialization needed
+    // ...
+  }
+
+  private void flush()

Review comment:
       in this case we can just throw an exception? rather than invoking flush in snapshotState? if it is not use anyway. 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] walterddr commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
walterddr commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r815384092



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/HttpClient.java
##########
@@ -0,0 +1,130 @@
+/**

Review comment:
       hmm. I must have missed something. here is what I did:
   1. deleted the entire folder under `src/main/java/org/apache/pinot/connector/flink/http`.
   2. run `mvn test -pl pinot-connectors/pinot-flink-connector`
   
   everything runs pass. this proofs that the http folder has nothing to do with this PR. yes?
   
   did you forget to add a Quickstart? I don't see a quickstart in this PR or any changes outside of the pinot-flink-connector module. (and if so, let's add it in a separate PR i suggest)




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (657daa5) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `7.02%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     71.06%   64.03%   -7.03%     
   + Complexity     4320     4240      -80     
   ============================================
     Files          1626     1586      -40     
     Lines         85036    83399    -1637     
     Branches      12795    12641     -154     
   ============================================
   - Hits          60428    53403    -7025     
   - Misses        20456    26147    +5691     
   + Partials       4152     3849     -303     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `66.96% <ø> (-0.39%)` | :arrow_down: |
   | unittests2 | `14.10% <ø> (+0.03%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ache/pinot/server/access/AccessControlFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VydmVyL2FjY2Vzcy9BY2Nlc3NDb250cm9sRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/SegmentReloadMessage.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvU2VnbWVudFJlbG9hZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/core/data/manager/realtime/TimerService.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvVGltZXJTZXJ2aWNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...che/pinot/controller/util/TableMetadataReader.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci91dGlsL1RhYmxlTWV0YWRhdGFSZWFkZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/minion/exception/TaskCancelledException.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vZXhjZXB0aW9uL1Rhc2tDYW5jZWxsZWRFeGNlcHRpb24uamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...not/common/exception/HttpErrorStatusException.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL0h0dHBFcnJvclN0YXR1c0V4Y2VwdGlvbi5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [422 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...657daa5](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter commented on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2a55ab2) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `56.94%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #8233       +/-   ##
   =============================================
   - Coverage     71.06%   14.12%   -56.95%     
   + Complexity     4320       81     -4239     
   =============================================
     Files          1626     1581       -45     
     Lines         85036    83194     -1842     
     Branches      12795    12598      -197     
   =============================================
   - Hits          60428    11748    -48680     
   - Misses        20456    70573    +50117     
   + Partials       4152      873     -3279     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `14.12% <ø> (+0.04%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ain/java/org/apache/pinot/core/data/table/Key.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL0tleS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/spi/utils/BooleanUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQm9vbGVhblV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/data/table/Record.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1JlY29yZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/util/GroupByUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dyb3VwQnlVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/config/table/FSTType.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL0ZTVFR5cGUuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/data/MetricFieldSpec.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9NZXRyaWNGaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/spi/utils/BigDecimalUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQmlnRGVjaW1hbFV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/common/tier/TierFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdGllci9UaWVyRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/common/utils/StringUtil.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvU3RyaW5nVXRpbC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/core/data/table/BaseTable.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL0Jhc2VUYWJsZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1297 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...2a55ab2](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2a55ab2) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `1.08%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     71.06%   69.97%   -1.09%     
   + Complexity     4320     4319       -1     
   ============================================
     Files          1626     1626              
     Lines         85036    85076      +40     
     Branches      12795    12802       +7     
   ============================================
   - Hits          60428    59533     -895     
   - Misses        20456    21421     +965     
   + Partials       4152     4122      -30     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `28.76% <ø> (-0.09%)` | :arrow_down: |
   | integration2 | `?` | |
   | unittests1 | `67.35% <ø> (+<0.01%)` | :arrow_up: |
   | unittests2 | `14.12% <ø> (+0.04%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...t/core/plan/StreamingInstanceResponsePlanNode.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9wbGFuL1N0cmVhbWluZ0luc3RhbmNlUmVzcG9uc2VQbGFuTm9kZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ore/operator/streaming/StreamingResponseUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9zdHJlYW1pbmcvU3RyZWFtaW5nUmVzcG9uc2VVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ager/realtime/PeerSchemeSplitSegmentCommitter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUGVlclNjaGVtZVNwbGl0U2VnbWVudENvbW1pdHRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/core/plan/StreamingSelectionPlanNode.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9wbGFuL1N0cmVhbWluZ1NlbGVjdGlvblBsYW5Ob2RlLmphdmE=) | `0.00% <0.00%> (-88.89%)` | :arrow_down: |
   | [...ator/streaming/StreamingSelectionOnlyOperator.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9zdHJlYW1pbmcvU3RyZWFtaW5nU2VsZWN0aW9uT25seU9wZXJhdG9yLmphdmE=) | `0.00% <0.00%> (-87.81%)` | :arrow_down: |
   | [...re/query/reduce/SelectionOnlyStreamingReducer.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvU2VsZWN0aW9uT25seVN0cmVhbWluZ1JlZHVjZXIuamF2YQ==) | `0.00% <0.00%> (-85.72%)` | :arrow_down: |
   | [...pache/pinot/common/utils/grpc/GrpcQueryClient.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvZ3JwYy9HcnBjUXVlcnlDbGllbnQuamF2YQ==) | `0.00% <0.00%> (-80.86%)` | :arrow_down: |
   | [...data/manager/realtime/DefaultSegmentCommitter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvRGVmYXVsdFNlZ21lbnRDb21taXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-80.00%)` | :arrow_down: |
   | [...ller/api/access/BasicAuthAccessControlFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvYWNjZXNzL0Jhc2ljQXV0aEFjY2Vzc0NvbnRyb2xGYWN0b3J5LmphdmE=) | `0.00% <0.00%> (-80.00%)` | :arrow_down: |
   | [...roker/requesthandler/GrpcBrokerRequestHandler.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcmVxdWVzdGhhbmRsZXIvR3JwY0Jyb2tlclJlcXVlc3RIYW5kbGVyLmphdmE=) | `0.00% <0.00%> (-78.58%)` | :arrow_down: |
   | ... and [91 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...2a55ab2](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] xiangfu0 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r812483384



##########
File path: pinot-connectors/pinot-flink-connector/src/test/resources/fixtures/pinotTableSchema.json
##########
@@ -0,0 +1,59 @@
+{
+  "schemaName": "demand",
+  "dimensionFieldSpecs": [
+    {
+      "name": "demand_uuid",
+      "singleValueField": true,
+      "dataType": "STRING",
+      "virtualColumnProvider": null,
+      "defaultNullValue": "null",
+      "transformFunction": null,
+      "maxLength" : 4096
+    },
+    {
+      "name": "geofence",
+      "singleValueField": true,
+      "dataType": "STRING",
+      "virtualColumnProvider": null,
+      "defaultNullValue": "null",
+      "transformFunction": null
+    }
+  ],
+  "metricFieldSpecs": [
+    {
+      "dataType": "DOUBLE",
+      "singleValueField": true,
+      "fieldSize": 8,
+      "derivedMetricType": null,
+      "name": "surge_multiplier",
+      "virtualColumnProvider": null,
+      "defaultNullValue": 0.0,
+      "transformFunction": null
+    }
+  ],
+  "dateTimeFieldSpecs": [
+  ],
+  "timeFieldSpec": {
+    "name": "timestamp",
+    "dataType": "LONG",
+    "singleValueField": true,
+    "incomingGranularitySpec": {
+      "name": "timestamp",
+      "dataType": "LONG",
+      "timeType": "MILLISECONDS",
+      "timeUnitSize": 1,
+      "timeFormat": "EPOCH"
+    },
+    "outgoingGranularitySpec": {
+      "name": "timestamp",
+      "dataType": "LONG",
+      "timeType": "MILLISECONDS",
+      "timeUnitSize": 1,
+      "timeFormat": "EPOCH"
+    },
+    "virtualColumnProvider": null,
+    "defaultNullValue": -9223372036854775808,
+    "transformFunction": null
+  },
+  "primaryKeyColumns" : [ "demand_uuid" ]
+}

Review comment:
       nit: new line, same for other files




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] xiangfu0 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r812492566



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/HttpClient.java
##########
@@ -0,0 +1,130 @@
+/**
+ * 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.connector.flink.http;
+
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import org.glassfish.jersey.client.JerseyClientBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/** This is copied from {@link HttpClient} in RTA-UMS project. */
+public abstract class HttpClient {

Review comment:
       Do we really need this abstract? I feel this class is more like static util methods if making `_httpClient` static.
   Also, we have implementations at `AbstractBaseAdminCommand` in `pinot-tools` for the same purpose, you can also consider moving this logic to pinot-common :p 
   
   




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] walterddr commented on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
walterddr commented on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1048441616


   this looks super great. thanks @yupeng9 for the contribution. I will take a look this weekend thoroughly


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r815369593



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/HttpClient.java
##########
@@ -0,0 +1,130 @@
+/**

Review comment:
       It's used in the quickstart to show how to use REST API to fetch the schema and table config to config the connector. Otherwise developer needs to manually craft those configs. 
   
   Also, you can take a look at `PinotConnectionUtils` for the useful config management.

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/PinotSinkFunction.java
##########
@@ -0,0 +1,169 @@
+/**
+ * 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.connector.flink.sink;
+
+import java.net.URI;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.pinot.connector.flink.common.RecordConverter;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.ingestion.segment.uploader.SegmentUploader;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The sink function for Pinot.
+ *
+ * <p>This version of the sink function doesn't leverage {@link SegmentWriter} API's ability buffer
+ * data and also share that data with checkpoint state. Instead it uses an internal buffer within
+ * PinotSinkFunction for checkpoint.
+ *
+ * <p>This should change once we introduce FlinkPinotSegmentWriter
+ *
+ * @param <T> type of record supported
+ */
+@SuppressWarnings("NullAway")
+public class PinotSinkFunction<T> extends RichSinkFunction<T> implements CheckpointedFunction {
+
+  public static final long DEFAULT_SEGMENT_FLUSH_MAX_NUM_RECORDS = 500000;
+  public static final int DEFAULT_EXECUTOR_POOL_SIZE = 5;
+  public static final long DEFAULT_EXECUTOR_SHUTDOWN_WAIT_MS = 3000;
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(PinotSinkFunction.class);
+
+  private final long _segmentFlushMaxNumRecords;
+  private final int _executorPoolSize;
+
+  private final RecordConverter<T> _recordConverter;
+
+  private TableConfig _tableConfig;
+  private Schema _schema;
+
+  private transient SegmentWriter _segmentWriter;
+  private transient SegmentUploader _segmentUploader;
+  private transient ExecutorService _executor;
+  private transient long _segmentNumRecord;
+
+  public PinotSinkFunction(RecordConverter<T> recordConverter, TableConfig tableConfig, Schema schema) {
+    this(recordConverter, tableConfig, schema, DEFAULT_SEGMENT_FLUSH_MAX_NUM_RECORDS, DEFAULT_EXECUTOR_POOL_SIZE);
+  }
+
+  public PinotSinkFunction(RecordConverter<T> recordConverter, TableConfig tableConfig, Schema schema,
+      long segmentFlushMaxNumRecords, int executorPoolSize) {
+    _recordConverter = recordConverter;
+    _tableConfig = tableConfig;
+    _schema = schema;
+    _segmentFlushMaxNumRecords = segmentFlushMaxNumRecords;
+    _executorPoolSize = executorPoolSize;
+  }
+
+  @Override
+  public void open(Configuration parameters)
+      throws Exception {
+    int indexOfSubtask = this.getRuntimeContext().getIndexOfThisSubtask();
+    // TODO improve segment uploader to use in-memory buffer then flush to tar file.
+    _segmentWriter = new FlinkSegmentWriter(indexOfSubtask, getRuntimeContext().getMetricGroup());
+    _segmentWriter.init(_tableConfig, _schema);
+    // TODO improve segment uploader to take in-memory tar
+    // TODO launch segment uploader as separate thread for uploading (non-blocking?)
+    _segmentUploader = new FlinkSegmentUploader(indexOfSubtask);
+    _segmentUploader.init(_tableConfig);
+    _segmentNumRecord = 0;
+    _executor = Executors.newFixedThreadPool(_executorPoolSize);
+    LOG.info("Open Pinot Sink with the table {}", _tableConfig.toJsonString());
+  }
+
+  @Override
+  public void close()
+      throws Exception {
+    LOG.info("Closing Pinot Sink");
+    try {
+      if (_segmentNumRecord > 0) {
+        flush();
+      }
+    } catch (Exception e) {
+      LOG.error("Error when closing Pinot sink", e);
+    }
+    _executor.shutdown();
+    try {
+      if (!_executor.awaitTermination(DEFAULT_EXECUTOR_SHUTDOWN_WAIT_MS, TimeUnit.MILLISECONDS)) {
+        _executor.shutdownNow();
+      }
+    } catch (InterruptedException e) {
+      _executor.shutdownNow();
+    }
+    _segmentWriter.close();
+  }
+
+  @Override
+  public void invoke(T value, Context context)
+      throws Exception {
+    _segmentWriter.collect(_recordConverter.convertToRow(value));
+    _segmentNumRecord++;
+    if (_segmentNumRecord > _segmentFlushMaxNumRecords) {
+      flush();
+    }
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext functionSnapshotContext)
+      throws Exception {
+    // TODO: not supported yet
+    LOG.error("snapshotState is invoked in Pinot sink");
+    // clear and flush.
+    flush();
+    // snapshot state:
+    // 1. should only work on the boundary of segment uploader.
+    // 2. segmentwriter state should be preserved.
+    // 3.
+    // ...
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext functionInitializationContext)
+      throws Exception {
+    // no initialization needed
+    // ...
+  }
+
+  private void flush()

Review comment:
       Is it possible to have a race conditon? Only `invoke` can call it, since `snapshotState` is not used in batch processing.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] walterddr commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
walterddr commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r815377380



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/HttpClient.java
##########
@@ -0,0 +1,130 @@
+/**

Review comment:
       yes, but it is not related to the flink connector. 
   
   for the sake of this PR's scope. i would suggest we drop it and add in separate PR.
   
   mainly for the following reasons. 
   - it doesn't relate to flink connector, at first I thought we are utilizing http to directly ingest into pinot via some sort of REST API. but if it is not the case it shouldn't be part of flink-pinot connector but rather in pinot-tools. 
   - @xiangfu0 also think we can refactor this part out later. doesn't make sense to get this in an refactor it out if it is not part of the critical path
   - there's already one util in test call ControllerUtils that can be used for demonstration purpose.
   
   Not saying it is not useful, but it would be much easier to address it separately. 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] xiangfu0 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r812492566



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/HttpClient.java
##########
@@ -0,0 +1,130 @@
+/**
+ * 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.connector.flink.http;
+
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import org.glassfish.jersey.client.JerseyClientBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/** This is copied from {@link HttpClient} in RTA-UMS project. */
+public abstract class HttpClient {

Review comment:
       Do we really need this abstract? I feel this class is more like static util if making `_httpClient` static.
   Also we have same implementations at `AbstractBaseAdminCommand` in `pinot-tools`.
   
   




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] walterddr commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
walterddr commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r812552807



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/PinotControllerClient.java
##########
@@ -0,0 +1,147 @@
+/**
+ * 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.connector.flink.http;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonValue;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/** This file is partially copied from RTAUMS gateway package */
+public class PinotControllerClient extends HttpClient {

Review comment:
       TBH we already have something very similar but is only used in test: ControllerTestUtils. we can probably consider a prime version of it 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r814319039



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/PinotControllerClient.java
##########
@@ -0,0 +1,147 @@
+/**
+ * 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.connector.flink.http;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonValue;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/** This file is partially copied from RTAUMS gateway package */
+public class PinotControllerClient extends HttpClient {

Review comment:
       sure. tracked at https://github.com/apache/pinot/issues/8246




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (657daa5) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `56.95%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #8233       +/-   ##
   =============================================
   - Coverage     71.06%   14.10%   -56.96%     
   + Complexity     4320       81     -4239     
   =============================================
     Files          1626     1586       -40     
     Lines         85036    83399     -1637     
     Branches      12795    12641      -154     
   =============================================
   - Hits          60428    11766    -48662     
   - Misses        20456    70754    +50298     
   + Partials       4152      879     -3273     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `14.10% <ø> (+0.03%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ain/java/org/apache/pinot/core/data/table/Key.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL0tleS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/spi/utils/BooleanUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQm9vbGVhblV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/data/table/Record.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1JlY29yZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/util/GroupByUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dyb3VwQnlVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/config/table/FSTType.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL0ZTVFR5cGUuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/data/MetricFieldSpec.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9NZXRyaWNGaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/spi/utils/BigDecimalUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQmlnRGVjaW1hbFV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/common/tier/TierFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdGllci9UaWVyRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/common/utils/StringUtil.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvU3RyaW5nVXRpbC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/core/data/table/BaseTable.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL0Jhc2VUYWJsZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1307 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...657daa5](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r819142901



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/FlinkQuickStart.java
##########
@@ -0,0 +1,88 @@
+/**
+ * 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.connector.flink;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.pinot.connector.flink.common.PinotRowRecordConverter;
+import org.apache.pinot.connector.flink.http.PinotConnectionUtils;
+import org.apache.pinot.connector.flink.http.PinotControllerClient;
+import org.apache.pinot.connector.flink.sink.PinotSinkFunction;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+
+
+/**
+ * A quick start to populate a segment into Pinot Table using the connector. Please run the GenericQuickStart to create
+ * the offline table of all Starbucks store locations in US, and then run this quick start to populate other Starbucks
+ * stores in the rest of the world.
+ */
+public final class FlinkQuickStart {

Review comment:
       not sure. pinot-tools doesn't have dependency on connectors. actually we do not build connectors for the core

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,266 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a buffer. The {@link GenericRow} are written to
+ * the buffer as AVRO records.
+ */
+@SuppressWarnings("NullAway")
+@NotThreadSafe
+public class FlinkSegmentWriter implements SegmentWriter {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FlinkSegmentWriter.class);
+  private static final FileFormat BUFFER_FILE_FORMAT = FileFormat.AVRO;
+
+  private final int _indexOfSubtask;
+
+  private TableConfig _tableConfig;
+  private String _tableNameWithType;
+  private BatchIngestionConfig _batchIngestionConfig;
+  private BatchConfig _batchConfig;
+  private String _outputDirURI;
+  private Schema _schema;
+  private Set<String> _fieldsToRead;
+  private RecordTransformer _recordTransformer;
+
+  private File _stagingDir;
+  private File _bufferFile;
+  private int _rowCount;
+  /** A sequence ID that increments each time a segment is flushed */
+  private int _seqId;
+
+  private org.apache.avro.Schema _avroSchema;
+  private DataFileWriter<GenericData.Record> _recordWriter;
+  private GenericData.Record _reusableRecord;
+
+  // metrics
+  private transient Counter _processedRecords;
+  private transient volatile long _lastRecordProcessingTimeMs = 0;
+
+  public FlinkSegmentWriter(int indexOfSubtask, MetricGroup metricGrp) {
+    _indexOfSubtask = indexOfSubtask;
+    registerMetrics(metricGrp);
+  }
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema)
+      throws Exception {
+    init(tableConfig, schema, null);
+  }
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema, Map<String, String> batchConfigOverride)
+      throws Exception {
+    _rowCount = 0;
+    _seqId = 0;
+    _tableConfig = tableConfig;
+    _tableNameWithType = _tableConfig.getTableName();
+    Preconditions.checkState(
+        _tableConfig.getIngestionConfig() != null && _tableConfig.getIngestionConfig().getBatchIngestionConfig() != null
+            && CollectionUtils.isNotEmpty(
+            _tableConfig.getIngestionConfig().getBatchIngestionConfig().getBatchConfigMaps()),
+        "Must provide ingestionConfig->batchIngestionConfig->batchConfigMaps in tableConfig for table: %s",
+        _tableNameWithType);
+    _batchIngestionConfig = _tableConfig.getIngestionConfig().getBatchIngestionConfig();
+    Preconditions.checkState(_batchIngestionConfig.getBatchConfigMaps().size() == 1,
+        "batchConfigMaps must contain only 1 BatchConfig for table: %s", _tableNameWithType);
+
+    Map<String, String> batchConfigMap = _batchIngestionConfig.getBatchConfigMaps().get(0);
+    String segmentNamePostfixProp = String.format("%s.%s", BatchConfigProperties.SEGMENT_NAME_GENERATOR_PROP_PREFIX,
+        BatchConfigProperties.SEGMENT_NAME_POSTFIX);
+    String segmentSuffix = batchConfigMap.get(segmentNamePostfixProp);
+    segmentSuffix = segmentSuffix == null ? String.valueOf(_indexOfSubtask) : segmentSuffix + "_" + _indexOfSubtask;
+    batchConfigMap.put(segmentNamePostfixProp, segmentSuffix);
+
+    _batchConfig = new BatchConfig(_tableNameWithType, batchConfigMap);
+
+    Preconditions.checkState(StringUtils.isNotBlank(_batchConfig.getOutputDirURI()),
+        "Must provide: %s in batchConfigs for table: %s", BatchConfigProperties.OUTPUT_DIR_URI, _tableNameWithType);
+    _outputDirURI = _batchConfig.getOutputDirURI();
+    Files.createDirectories(Paths.get(_outputDirURI));
+
+    _schema = schema;
+    _fieldsToRead = _schema.getColumnNames();
+    _recordTransformer = CompositeTransformer.getDefaultTransformer(_tableConfig, _schema);
+    _avroSchema = SegmentProcessorAvroUtils.convertPinotSchemaToAvroSchema(_schema);
+    _reusableRecord = new GenericData.Record(_avroSchema);
+
+    // Create tmp dir
+    _stagingDir = new File(FileUtils.getTempDirectory(),
+        String.format("segment_writer_staging_%s_%d_%d", _tableNameWithType, _indexOfSubtask,
+            System.currentTimeMillis()));
+    Preconditions.checkState(_stagingDir.mkdirs(), "Failed to create staging dir: %s", _stagingDir.getAbsolutePath());
+
+    // Create buffer file
+    File bufferDir = new File(_stagingDir, "buffer_dir");
+    Preconditions.checkState(bufferDir.mkdirs(), "Failed to create buffer_dir: %s", bufferDir.getAbsolutePath());
+    _bufferFile = new File(bufferDir, "buffer_file");
+    resetBuffer();
+    LOGGER.info("Initialized {} for Pinot table: {}", this.getClass().getName(), _tableNameWithType);
+  }
+
+  private void registerMetrics(MetricGroup metricGrp) {
+    _processedRecords = metricGrp.counter("records.processed");
+    metricGrp.gauge("record.processing.time.ts", (Gauge<Long>) () -> _lastRecordProcessingTimeMs);
+  }
+
+  private void resetBuffer()
+      throws IOException {
+    FileUtils.deleteQuietly(_bufferFile);
+    _rowCount = 0;
+    _recordWriter = new DataFileWriter<>(new GenericDatumWriter<>(_avroSchema));
+    _recordWriter.create(_avroSchema, _bufferFile);
+  }
+
+  @Override
+  public void collect(GenericRow row)
+      throws IOException {
+    long startTime = System.currentTimeMillis();
+    GenericRow transform = _recordTransformer.transform(row);
+    SegmentProcessorAvroUtils.convertGenericRowToAvroRecord(transform, _reusableRecord, _fieldsToRead);
+    _rowCount++;
+    _recordWriter.append(_reusableRecord);
+    _lastRecordProcessingTimeMs = System.currentTimeMillis() - startTime;
+    _processedRecords.inc();
+  }
+
+  /**
+   * Creates one Pinot segment using the {@link GenericRow}s collected in the AVRO file buffer, at
+   * the outputDirUri as specified in the tableConfig->batchConfigs. Successful invocation of this
+   * method means that the {@link GenericRow}s collected so far, are now available in the Pinot
+   * segment and not available in the buffer anymore.
+   *
+   * <p>Successful completion of segment will return the segment URI, and the URI includes a
+   * sequence id indicating the part number. The sequence id is initialized to 0 and each successful
+   * flush will increment the sequence id by 1. The buffer will be reset and ready to accept further
+   * records via <code>collect()</code> If an exception is thrown, the buffer will not be reset and
+   * so, <code>flush()</code> can be invoked repeatedly in a retry loop. If a successful invocation
+   * is not achieved,<code>close()</code> followed by <code>init </code> will have to be called in
+   * order to reset the buffer and resume record writing.
+   *
+   * @return URI of the generated segment
+   * @throws IOException
+   */
+  @Override
+  public URI flush()
+      throws IOException {
+
+    LOGGER.info("Beginning flush for Pinot table: {} with {} records", _tableNameWithType, _rowCount);
+    _recordWriter.close();
+
+    // Create temp dir for flush
+    File flushDir = new File(_stagingDir, "flush_dir_" + System.currentTimeMillis());
+    Preconditions.checkState(flushDir.mkdirs(), "Failed to create flush dir: %s", flushDir);
+
+    try {
+      // Segment dir
+      File segmentDir = new File(flushDir, "segment_dir");
+
+      // Make BatchIngestionConfig for flush
+      Map<String, String> batchConfigMapOverride = new HashMap<>(_batchConfig.getBatchConfigMap());
+      batchConfigMapOverride.put(BatchConfigProperties.INPUT_DIR_URI, _bufferFile.getAbsolutePath());
+      batchConfigMapOverride.put(BatchConfigProperties.OUTPUT_DIR_URI, segmentDir.getAbsolutePath());
+      batchConfigMapOverride.put(BatchConfigProperties.INPUT_FORMAT, BUFFER_FILE_FORMAT.toString());
+      BatchIngestionConfig batchIngestionConfig = new BatchIngestionConfig(Lists.newArrayList(batchConfigMapOverride),
+          _batchIngestionConfig.getSegmentIngestionType(), _batchIngestionConfig.getSegmentIngestionFrequency());
+
+      // Build segment
+      SegmentGeneratorConfig segmentGeneratorConfig =
+          IngestionUtils.generateSegmentGeneratorConfig(_tableConfig, _schema, batchIngestionConfig);
+      segmentGeneratorConfig.setSequenceId(_seqId);
+      String segmentName = IngestionUtils.buildSegment(segmentGeneratorConfig);
+      LOGGER.info("Successfully built segment: {} of sequence {} for Pinot table: {}", segmentName, _seqId,
+          _tableNameWithType);
+
+      // Tar segment
+      File segmentTarFile =
+          new File(_outputDirURI, String.format("%s_%d%s", segmentName, _indexOfSubtask, Constants.TAR_GZ_FILE_EXT));

Review comment:
       do you mean `_indexOfSubtask`? that's different from the sequence id, `_indexOfSubtask` is the id of the Flink task.

##########
File path: pinot-connectors/pinot-flink-connector/README.md
##########
@@ -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.
+
+-->
+# Flink-Pinot Connector
+
+Flink connector to write data to Pinot directly. This is useful for backfilling or bootstrapping tables,
+including the upsert tables. You can read more about the motivation and design in this [design proposal](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=177045634).
+
+## Quick Start
+```java
+StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+execEnv.setParallelism(2);
+DataStream<Row> srcDs = execEnv.fromCollection(data).returns(TEST_TYPE_INFO).keyBy(r -> r.getField(0));
+
+PinotControllerClient client = new PinotControllerClient();
+// fetch Pinot schema
+Schema schema = PinotConnectionUtils.getSchema(client, "starbucksStores");
+// fetch Pinot table config
+TableConfig tableConfig = PinotConnectionUtils.getTableConfig(client, "starbucksStores", "OFFLINE");
+// create Flink Pinot Sink
+srcDs.addSink(new PinotSinkFunction<>(new PinotRowRecordConverter(TEST_TYPE_INFO), tableConfig, schema));
+execEnv.execute();
+```
+
+For more examples, please see `src/main/java/org/apache/pinot/connector/flink/FlinkQuickStart.java`
+
+## Notes for backfilling upsert table
+ - To correctly partition the output segments by the primary key, the Flink job *must* also include the partitionBeyKey operator before the Sink operator
+ - The parallelism of the job *must* be set the same as the number of partitions of the Pinot table, so that the sink in each task executor can generate the segment of same partitions.
+ - It’s important to plan the resource usage to avoid capacity issues such as out of memory. In particular, Pinot sink has an in-memory buffer of records, and it flushes when the threshold is reached. Currently, the threshold on the number of records is supported via the config of `segmentFlushMaxNumRecords`. In the future, we could add other types of threshold such as the memory usage of the buffer.

Review comment:
       I think `DataFileWriter` keeps an in-mem buffer, and flush to disk upon `flush`.  And we do see the sink can run out of memory if not flushed upon threshold.

##########
File path: pinot-connectors/pinot-flink-connector/README.md
##########
@@ -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.
+
+-->
+# Flink-Pinot Connector
+
+Flink connector to write data to Pinot directly. This is useful for backfilling or bootstrapping tables,
+including the upsert tables. You can read more about the motivation and design in this [design proposal](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=177045634).
+
+## Quick Start
+```java
+StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+execEnv.setParallelism(2);
+DataStream<Row> srcDs = execEnv.fromCollection(data).returns(TEST_TYPE_INFO).keyBy(r -> r.getField(0));
+
+PinotControllerClient client = new PinotControllerClient();
+// fetch Pinot schema
+Schema schema = PinotConnectionUtils.getSchema(client, "starbucksStores");
+// fetch Pinot table config
+TableConfig tableConfig = PinotConnectionUtils.getTableConfig(client, "starbucksStores", "OFFLINE");
+// create Flink Pinot Sink
+srcDs.addSink(new PinotSinkFunction<>(new PinotRowRecordConverter(TEST_TYPE_INFO), tableConfig, schema));
+execEnv.execute();
+```
+
+For more examples, please see `src/main/java/org/apache/pinot/connector/flink/FlinkQuickStart.java`
+
+## Notes for backfilling upsert table
+ - To correctly partition the output segments by the primary key, the Flink job *must* also include the partitionBeyKey operator before the Sink operator

Review comment:
       good catch. even spell checker didn't catch it :)

##########
File path: pinot-connectors/pinot-flink-connector/src/test/java/org/apache/pinot/connector/flink/util/TestUtils.java
##########
@@ -0,0 +1,44 @@
+/**
+ * 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.connector.flink.util;
+
+import java.io.File;
+import java.io.IOException;
+import javax.annotation.Nonnull;
+import org.apache.commons.io.FileUtils;
+import org.testng.Assert;
+
+
+public final class TestUtils {

Review comment:
       `pinot-integration-tests` does not have dependency on connectors.

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,266 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a buffer. The {@link GenericRow} are written to
+ * the buffer as AVRO records.
+ */
+@SuppressWarnings("NullAway")
+@NotThreadSafe
+public class FlinkSegmentWriter implements SegmentWriter {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FlinkSegmentWriter.class);
+  private static final FileFormat BUFFER_FILE_FORMAT = FileFormat.AVRO;
+
+  private final int _indexOfSubtask;
+
+  private TableConfig _tableConfig;
+  private String _tableNameWithType;
+  private BatchIngestionConfig _batchIngestionConfig;
+  private BatchConfig _batchConfig;
+  private String _outputDirURI;
+  private Schema _schema;
+  private Set<String> _fieldsToRead;
+  private RecordTransformer _recordTransformer;
+
+  private File _stagingDir;
+  private File _bufferFile;
+  private int _rowCount;
+  /** A sequence ID that increments each time a segment is flushed */
+  private int _seqId;
+
+  private org.apache.avro.Schema _avroSchema;
+  private DataFileWriter<GenericData.Record> _recordWriter;
+  private GenericData.Record _reusableRecord;
+
+  // metrics
+  private transient Counter _processedRecords;
+  private transient volatile long _lastRecordProcessingTimeMs = 0;
+
+  public FlinkSegmentWriter(int indexOfSubtask, MetricGroup metricGrp) {
+    _indexOfSubtask = indexOfSubtask;
+    registerMetrics(metricGrp);
+  }
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema)
+      throws Exception {
+    init(tableConfig, schema, null);
+  }
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema, Map<String, String> batchConfigOverride)
+      throws Exception {
+    _rowCount = 0;
+    _seqId = 0;
+    _tableConfig = tableConfig;
+    _tableNameWithType = _tableConfig.getTableName();
+    Preconditions.checkState(
+        _tableConfig.getIngestionConfig() != null && _tableConfig.getIngestionConfig().getBatchIngestionConfig() != null
+            && CollectionUtils.isNotEmpty(
+            _tableConfig.getIngestionConfig().getBatchIngestionConfig().getBatchConfigMaps()),
+        "Must provide ingestionConfig->batchIngestionConfig->batchConfigMaps in tableConfig for table: %s",
+        _tableNameWithType);
+    _batchIngestionConfig = _tableConfig.getIngestionConfig().getBatchIngestionConfig();
+    Preconditions.checkState(_batchIngestionConfig.getBatchConfigMaps().size() == 1,
+        "batchConfigMaps must contain only 1 BatchConfig for table: %s", _tableNameWithType);
+
+    Map<String, String> batchConfigMap = _batchIngestionConfig.getBatchConfigMaps().get(0);
+    String segmentNamePostfixProp = String.format("%s.%s", BatchConfigProperties.SEGMENT_NAME_GENERATOR_PROP_PREFIX,
+        BatchConfigProperties.SEGMENT_NAME_POSTFIX);
+    String segmentSuffix = batchConfigMap.get(segmentNamePostfixProp);
+    segmentSuffix = segmentSuffix == null ? String.valueOf(_indexOfSubtask) : segmentSuffix + "_" + _indexOfSubtask;
+    batchConfigMap.put(segmentNamePostfixProp, segmentSuffix);
+
+    _batchConfig = new BatchConfig(_tableNameWithType, batchConfigMap);
+
+    Preconditions.checkState(StringUtils.isNotBlank(_batchConfig.getOutputDirURI()),
+        "Must provide: %s in batchConfigs for table: %s", BatchConfigProperties.OUTPUT_DIR_URI, _tableNameWithType);
+    _outputDirURI = _batchConfig.getOutputDirURI();
+    Files.createDirectories(Paths.get(_outputDirURI));
+
+    _schema = schema;
+    _fieldsToRead = _schema.getColumnNames();
+    _recordTransformer = CompositeTransformer.getDefaultTransformer(_tableConfig, _schema);
+    _avroSchema = SegmentProcessorAvroUtils.convertPinotSchemaToAvroSchema(_schema);
+    _reusableRecord = new GenericData.Record(_avroSchema);
+
+    // Create tmp dir
+    _stagingDir = new File(FileUtils.getTempDirectory(),
+        String.format("segment_writer_staging_%s_%d_%d", _tableNameWithType, _indexOfSubtask,
+            System.currentTimeMillis()));
+    Preconditions.checkState(_stagingDir.mkdirs(), "Failed to create staging dir: %s", _stagingDir.getAbsolutePath());
+
+    // Create buffer file
+    File bufferDir = new File(_stagingDir, "buffer_dir");
+    Preconditions.checkState(bufferDir.mkdirs(), "Failed to create buffer_dir: %s", bufferDir.getAbsolutePath());
+    _bufferFile = new File(bufferDir, "buffer_file");
+    resetBuffer();
+    LOGGER.info("Initialized {} for Pinot table: {}", this.getClass().getName(), _tableNameWithType);
+  }
+
+  private void registerMetrics(MetricGroup metricGrp) {
+    _processedRecords = metricGrp.counter("records.processed");
+    metricGrp.gauge("record.processing.time.ts", (Gauge<Long>) () -> _lastRecordProcessingTimeMs);
+  }
+
+  private void resetBuffer()
+      throws IOException {
+    FileUtils.deleteQuietly(_bufferFile);
+    _rowCount = 0;
+    _recordWriter = new DataFileWriter<>(new GenericDatumWriter<>(_avroSchema));
+    _recordWriter.create(_avroSchema, _bufferFile);
+  }
+
+  @Override
+  public void collect(GenericRow row)
+      throws IOException {
+    long startTime = System.currentTimeMillis();
+    GenericRow transform = _recordTransformer.transform(row);
+    SegmentProcessorAvroUtils.convertGenericRowToAvroRecord(transform, _reusableRecord, _fieldsToRead);
+    _rowCount++;
+    _recordWriter.append(_reusableRecord);
+    _lastRecordProcessingTimeMs = System.currentTimeMillis() - startTime;
+    _processedRecords.inc();
+  }
+
+  /**
+   * Creates one Pinot segment using the {@link GenericRow}s collected in the AVRO file buffer, at
+   * the outputDirUri as specified in the tableConfig->batchConfigs. Successful invocation of this
+   * method means that the {@link GenericRow}s collected so far, are now available in the Pinot
+   * segment and not available in the buffer anymore.
+   *
+   * <p>Successful completion of segment will return the segment URI, and the URI includes a
+   * sequence id indicating the part number. The sequence id is initialized to 0 and each successful
+   * flush will increment the sequence id by 1. The buffer will be reset and ready to accept further
+   * records via <code>collect()</code> If an exception is thrown, the buffer will not be reset and
+   * so, <code>flush()</code> can be invoked repeatedly in a retry loop. If a successful invocation
+   * is not achieved,<code>close()</code> followed by <code>init </code> will have to be called in
+   * order to reset the buffer and resume record writing.
+   *
+   * @return URI of the generated segment
+   * @throws IOException
+   */
+  @Override
+  public URI flush()
+      throws IOException {
+
+    LOGGER.info("Beginning flush for Pinot table: {} with {} records", _tableNameWithType, _rowCount);
+    _recordWriter.close();
+
+    // Create temp dir for flush
+    File flushDir = new File(_stagingDir, "flush_dir_" + System.currentTimeMillis());
+    Preconditions.checkState(flushDir.mkdirs(), "Failed to create flush dir: %s", flushDir);
+
+    try {
+      // Segment dir
+      File segmentDir = new File(flushDir, "segment_dir");
+
+      // Make BatchIngestionConfig for flush
+      Map<String, String> batchConfigMapOverride = new HashMap<>(_batchConfig.getBatchConfigMap());
+      batchConfigMapOverride.put(BatchConfigProperties.INPUT_DIR_URI, _bufferFile.getAbsolutePath());
+      batchConfigMapOverride.put(BatchConfigProperties.OUTPUT_DIR_URI, segmentDir.getAbsolutePath());
+      batchConfigMapOverride.put(BatchConfigProperties.INPUT_FORMAT, BUFFER_FILE_FORMAT.toString());
+      BatchIngestionConfig batchIngestionConfig = new BatchIngestionConfig(Lists.newArrayList(batchConfigMapOverride),
+          _batchIngestionConfig.getSegmentIngestionType(), _batchIngestionConfig.getSegmentIngestionFrequency());
+
+      // Build segment
+      SegmentGeneratorConfig segmentGeneratorConfig =
+          IngestionUtils.generateSegmentGeneratorConfig(_tableConfig, _schema, batchIngestionConfig);
+      segmentGeneratorConfig.setSequenceId(_seqId);
+      String segmentName = IngestionUtils.buildSegment(segmentGeneratorConfig);
+      LOGGER.info("Successfully built segment: {} of sequence {} for Pinot table: {}", segmentName, _seqId,
+          _tableNameWithType);
+
+      // Tar segment
+      File segmentTarFile =
+          new File(_outputDirURI, String.format("%s_%d%s", segmentName, _indexOfSubtask, Constants.TAR_GZ_FILE_EXT));
+      if (!_batchConfig.isOverwriteOutput() && segmentTarFile.exists()) {
+        segmentTarFile = new File(_outputDirURI,
+            String.format("%s_%d_%d%s", segmentName, _indexOfSubtask, System.currentTimeMillis(),
+                Constants.TAR_GZ_FILE_EXT));
+      }

Review comment:
       yes. it guarantees. Each segment in each run will have its own file name, and the file will be deleted after the run

##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,266 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a buffer. The {@link GenericRow} are written to
+ * the buffer as AVRO records.
+ */
+@SuppressWarnings("NullAway")
+@NotThreadSafe
+public class FlinkSegmentWriter implements SegmentWriter {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FlinkSegmentWriter.class);
+  private static final FileFormat BUFFER_FILE_FORMAT = FileFormat.AVRO;
+
+  private final int _indexOfSubtask;
+
+  private TableConfig _tableConfig;
+  private String _tableNameWithType;
+  private BatchIngestionConfig _batchIngestionConfig;
+  private BatchConfig _batchConfig;
+  private String _outputDirURI;
+  private Schema _schema;
+  private Set<String> _fieldsToRead;
+  private RecordTransformer _recordTransformer;
+
+  private File _stagingDir;
+  private File _bufferFile;
+  private int _rowCount;
+  /** A sequence ID that increments each time a segment is flushed */

Review comment:
       Right.There's a buffer limit, so we need to generate multiple segments and this sequence ID is to differentiate the generated segment files.

##########
File path: pinot-connectors/pinot-flink-connector/src/test/java/org/apache/pinot/connector/flink/util/TestUtils.java
##########
@@ -0,0 +1,44 @@
+/**
+ * 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.connector.flink.util;
+
+import java.io.File;
+import java.io.IOException;
+import javax.annotation.Nonnull;
+import org.apache.commons.io.FileUtils;
+import org.testng.Assert;
+
+
+public final class TestUtils {
+
+  private TestUtils() {
+  }
+
+  public static void ensureDirectoriesExistAndEmpty(@Nonnull File... dirs)
+      throws IOException {
+    File[] var1 = dirs;
+    int var2 = dirs.length;
+
+    for (int var3 = 0; var3 < var2; var3++) {

Review comment:
       hmm, interesting, when I copied this file from the uber internal repo, it seems I copied the decompiled source code...




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6eef992) into [master](https://codecov.io/gh/apache/pinot/commit/46ed731c4e60c308c9559e46349a984b0ce05ce6?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (46ed731) will **decrease** coverage by `8.39%`.
   > The diff coverage is `n/a`.
   
   > :exclamation: Current head 6eef992 differs from pull request most recent head 05d2c8e. Consider uploading reports for the commit 05d2c8e to get more accurate results
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     70.83%   62.43%   -8.40%     
   + Complexity     4245     4166      -79     
   ============================================
     Files          1631     1619      -12     
     Lines         85462    85108     -354     
     Branches      12877    12839      -38     
   ============================================
   - Hits          60539    53141    -7398     
   - Misses        20746    28045    +7299     
   + Partials       4177     3922     -255     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `28.95% <ø> (-0.04%)` | :arrow_down: |
   | integration2 | `?` | |
   | unittests1 | `67.00% <ø> (+0.04%)` | :arrow_up: |
   | unittests2 | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...t/core/plan/StreamingInstanceResponsePlanNode.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9wbGFuL1N0cmVhbWluZ0luc3RhbmNlUmVzcG9uc2VQbGFuTm9kZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/controller/recommender/io/ConfigManager.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9yZWNvbW1lbmRlci9pby9Db25maWdNYW5hZ2VyLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ore/operator/streaming/StreamingResponseUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9zdHJlYW1pbmcvU3RyZWFtaW5nUmVzcG9uc2VVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...troller/recommender/io/metadata/FieldMetadata.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9yZWNvbW1lbmRlci9pby9tZXRhZGF0YS9GaWVsZE1ldGFkYXRhLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...roller/recommender/rules/impl/BloomFilterRule.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9yZWNvbW1lbmRlci9ydWxlcy9pbXBsL0Jsb29tRmlsdGVyUnVsZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...oller/api/resources/PinotControllerAppConfigs.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL1Bpbm90Q29udHJvbGxlckFwcENvbmZpZ3MuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ler/recommender/data/generator/BytesGenerator.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9yZWNvbW1lbmRlci9kYXRhL2dlbmVyYXRvci9CeXRlc0dlbmVyYXRvci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ager/realtime/PeerSchemeSplitSegmentCommitter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUGVlclNjaGVtZVNwbGl0U2VnbWVudENvbW1pdHRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...er/recommender/io/metadata/SchemaWithMetaData.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9yZWNvbW1lbmRlci9pby9tZXRhZGF0YS9TY2hlbWFXaXRoTWV0YURhdGEuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...r/recommender/rules/impl/AggregateMetricsRule.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9yZWNvbW1lbmRlci9ydWxlcy9pbXBsL0FnZ3JlZ2F0ZU1ldHJpY3NSdWxlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [277 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [46ed731...05d2c8e](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r812550288



##########
File path: pinot-connectors/pinot-flink-connector/src/test/java/org/apache/pinot/connector/flink/http/PinotControllerClientTest.java
##########
@@ -0,0 +1,168 @@
+/**
+ * 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.connector.flink.http;
+
+import com.google.common.collect.Lists;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.Invocation;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedHashMap;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import org.apache.commons.io.IOUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.testng.annotations.Test;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.assertEquals;
+
+
+public class PinotControllerClientTest {
+
+  private static final String TABLE_NAME = "demand";
+
+  @Test
+  public void testGetPinotControllerInstancesFromController()
+      throws Exception {
+    Client client = mock(Client.class);
+    WebTarget target = mock(WebTarget.class);
+    Invocation.Builder builder = mock(Invocation.Builder.class);
+    Response response = mock(Response.class);
+    MultivaluedMap<String, Object> headers = getRequestHeadersToPinot();
+    String expectedFullURL = "http://localhost:9000/instances";
+    final Map<String, Object> resEntity =
+        JsonUtils.stringToObject(fixture("fixtures/pinotControllerInstances.json"), Map.class);
+
+    when(client.target(expectedFullURL)).thenReturn(target);
+    when(target.request()).thenReturn(builder);
+    when(builder.headers(headers)).thenReturn(builder);
+    when(builder.get()).thenReturn(response);
+    when(response.getStatus()).thenReturn(200);
+    when(response.getEntity()).thenReturn(resEntity);
+    when(response.readEntity(Map.class)).thenReturn(resEntity);
+
+    PinotControllerClient controllerClient = new PinotControllerClient();
+    controllerClient.setHttpClient(client);
+    List<String> instances = controllerClient.getControllerInstances(new MultivaluedHashMap<>());
+    assertEquals(Lists.newArrayList("pinot-prod02:5983"), instances);
+
+    verify(client, times(1)).target(expectedFullURL);
+    verify(target, times(1)).request();
+    verify(builder, times(1)).headers(headers);
+    verify(builder, times(1)).get();
+  }
+
+  @Test
+  public void testGetPinotSchemaStrFromController()
+      throws Exception {
+    Client client = mock(Client.class);
+    WebTarget target = mock(WebTarget.class);
+    Invocation.Builder builder = mock(Invocation.Builder.class);
+    Response response = mock(Response.class);
+    MultivaluedMap<String, Object> headers = getRequestHeadersToPinot();
+    String expectedFullURL = "http://localhost:9000/schemas/demand";
+    final Map<String, Object> resEntity =
+        JsonUtils.stringToObject(fixture("fixtures/pinotTableSchema.json"), Map.class);
+
+    when(client.target(expectedFullURL)).thenReturn(target);
+    when(target.request()).thenReturn(builder);
+    when(builder.headers(headers)).thenReturn(builder);
+    when(builder.get()).thenReturn(response);
+    when(response.getStatus()).thenReturn(200);
+    when(response.getEntity()).thenReturn(resEntity);
+    when(response.readEntity(Map.class)).thenReturn(resEntity);
+
+    PinotControllerClient controllerClient = new PinotControllerClient();
+    controllerClient.setHttpClient(client);
+    String schemaStrFromController =
+        controllerClient.getSchemaStrFromController(TABLE_NAME, new MultivaluedHashMap<>());
+    assertEquals(
+        JsonUtils.objectToString(JsonUtils.stringToObject(fixture("fixtures/pinotTableSchema.json"), Map.class)),
+        schemaStrFromController);
+
+    verify(client, times(1)).target(expectedFullURL);
+    verify(target, times(1)).request();
+    verify(builder, times(1)).headers(headers);
+    verify(builder, times(1)).get();
+  }
+
+  @Test
+  public void testGetPinotConfigStrFromController()
+      throws Exception {
+    Client client = mock(Client.class);
+    WebTarget target = mock(WebTarget.class);
+    Invocation.Builder builder = mock(Invocation.Builder.class);
+    Response response = mock(Response.class);
+    MultivaluedMap<String, Object> headers = getRequestHeadersToPinot();
+    String expectedFullURL = "http://localhost:9000/tables/demand?type=realtime";
+    final Map<String, Object> resEntity = new HashMap<>() {

Review comment:
       I thought we dropped jdk8 support? not yet?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] xiangfu0 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r812483234



##########
File path: pinot-connectors/pinot-flink-connector/src/test/java/org/apache/pinot/connector/flink/http/PinotControllerClientTest.java
##########
@@ -0,0 +1,168 @@
+/**
+ * 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.connector.flink.http;
+
+import com.google.common.collect.Lists;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.Invocation;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedHashMap;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import org.apache.commons.io.IOUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.testng.annotations.Test;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.assertEquals;
+
+
+public class PinotControllerClientTest {
+
+  private static final String TABLE_NAME = "demand";
+
+  @Test
+  public void testGetPinotControllerInstancesFromController()
+      throws Exception {
+    Client client = mock(Client.class);
+    WebTarget target = mock(WebTarget.class);
+    Invocation.Builder builder = mock(Invocation.Builder.class);
+    Response response = mock(Response.class);
+    MultivaluedMap<String, Object> headers = getRequestHeadersToPinot();
+    String expectedFullURL = "http://localhost:9000/instances";
+    final Map<String, Object> resEntity =
+        JsonUtils.stringToObject(fixture("fixtures/pinotControllerInstances.json"), Map.class);
+
+    when(client.target(expectedFullURL)).thenReturn(target);
+    when(target.request()).thenReturn(builder);
+    when(builder.headers(headers)).thenReturn(builder);
+    when(builder.get()).thenReturn(response);
+    when(response.getStatus()).thenReturn(200);
+    when(response.getEntity()).thenReturn(resEntity);
+    when(response.readEntity(Map.class)).thenReturn(resEntity);
+
+    PinotControllerClient controllerClient = new PinotControllerClient();
+    controllerClient.setHttpClient(client);
+    List<String> instances = controllerClient.getControllerInstances(new MultivaluedHashMap<>());
+    assertEquals(Lists.newArrayList("pinot-prod02:5983"), instances);
+
+    verify(client, times(1)).target(expectedFullURL);
+    verify(target, times(1)).request();
+    verify(builder, times(1)).headers(headers);
+    verify(builder, times(1)).get();
+  }
+
+  @Test
+  public void testGetPinotSchemaStrFromController()
+      throws Exception {
+    Client client = mock(Client.class);
+    WebTarget target = mock(WebTarget.class);
+    Invocation.Builder builder = mock(Invocation.Builder.class);
+    Response response = mock(Response.class);
+    MultivaluedMap<String, Object> headers = getRequestHeadersToPinot();
+    String expectedFullURL = "http://localhost:9000/schemas/demand";
+    final Map<String, Object> resEntity =
+        JsonUtils.stringToObject(fixture("fixtures/pinotTableSchema.json"), Map.class);
+
+    when(client.target(expectedFullURL)).thenReturn(target);
+    when(target.request()).thenReturn(builder);
+    when(builder.headers(headers)).thenReturn(builder);
+    when(builder.get()).thenReturn(response);
+    when(response.getStatus()).thenReturn(200);
+    when(response.getEntity()).thenReturn(resEntity);
+    when(response.readEntity(Map.class)).thenReturn(resEntity);
+
+    PinotControllerClient controllerClient = new PinotControllerClient();
+    controllerClient.setHttpClient(client);
+    String schemaStrFromController =
+        controllerClient.getSchemaStrFromController(TABLE_NAME, new MultivaluedHashMap<>());
+    assertEquals(
+        JsonUtils.objectToString(JsonUtils.stringToObject(fixture("fixtures/pinotTableSchema.json"), Map.class)),
+        schemaStrFromController);
+
+    verify(client, times(1)).target(expectedFullURL);
+    verify(target, times(1)).request();
+    verify(builder, times(1)).headers(headers);
+    verify(builder, times(1)).get();
+  }
+
+  @Test
+  public void testGetPinotConfigStrFromController()
+      throws Exception {
+    Client client = mock(Client.class);
+    WebTarget target = mock(WebTarget.class);
+    Invocation.Builder builder = mock(Invocation.Builder.class);
+    Response response = mock(Response.class);
+    MultivaluedMap<String, Object> headers = getRequestHeadersToPinot();
+    String expectedFullURL = "http://localhost:9000/tables/demand?type=realtime";
+    final Map<String, Object> resEntity = new HashMap<>() {

Review comment:
       This seems not compilable for jdk8:
   ```
   Error:  Failed to execute goal org.apache.maven.plugins:maven-compiler-plugin:3.8.0:testCompile (default-testCompile) on project flink-pinot-sink: Compilation failure
   Error:  /home/runner/work/pinot/pinot/pinot-connectors/pinot-flink-connector/src/test/java/org/apache/pinot/connector/flink/http/PinotControllerClientTest.java:[124,53] error: cannot infer type arguments for HashMap<K,V>
   Error:    reason: cannot use '<>' with anonymous inner classes
   Error:    where K,V are type-variables:
   Error:      K extends Object declared in class HashMap
   Error:      V extends Object declared in class HashMap
   Error:  -> [Help 1]
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (283b7c3) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `1.58%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     71.06%   69.47%   -1.59%     
   + Complexity     4320     4241      -79     
   ============================================
     Files          1626     1629       +3     
     Lines         85036    85152     +116     
     Branches      12795    12815      +20     
   ============================================
   - Hits          60428    59159    -1269     
   - Misses        20456    21872    +1416     
   + Partials       4152     4121      -31     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `27.49% <ø> (-0.05%)` | :arrow_down: |
   | unittests1 | `66.96% <ø> (-0.39%)` | :arrow_down: |
   | unittests2 | `14.11% <ø> (+0.03%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...pinot/minion/exception/TaskCancelledException.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vZXhjZXB0aW9uL1Rhc2tDYW5jZWxsZWRFeGNlcHRpb24uamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...nverttorawindex/ConvertToRawIndexTaskExecutor.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1taW5pb24tdGFza3MvcGlub3QtbWluaW9uLWJ1aWx0aW4tdGFza3Mvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3BsdWdpbi9taW5pb24vdGFza3MvY29udmVydHRvcmF3aW5kZXgvQ29udmVydFRvUmF3SW5kZXhUYXNrRXhlY3V0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...e/pinot/common/minion/MergeRollupTaskMetadata.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWluaW9uL01lcmdlUm9sbHVwVGFza01ldGFkYXRhLmphdmE=) | `0.00% <0.00%> (-94.74%)` | :arrow_down: |
   | [...plugin/segmentuploader/SegmentUploaderDefault.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1zZWdtZW50LXVwbG9hZGVyL3Bpbm90LXNlZ21lbnQtdXBsb2FkZXItZGVmYXVsdC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcGx1Z2luL3NlZ21lbnR1cGxvYWRlci9TZWdtZW50VXBsb2FkZXJEZWZhdWx0LmphdmE=) | `0.00% <0.00%> (-87.10%)` | :arrow_down: |
   | [.../transform/function/MapValueTransformFunction.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vTWFwVmFsdWVUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `0.00% <0.00%> (-85.30%)` | :arrow_down: |
   | [...ot/common/messages/RoutingTableRebuildMessage.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvUm91dGluZ1RhYmxlUmVidWlsZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-81.82%)` | :arrow_down: |
   | [...data/manager/realtime/DefaultSegmentCommitter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvRGVmYXVsdFNlZ21lbnRDb21taXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-80.00%)` | :arrow_down: |
   | [...verttorawindex/ConvertToRawIndexTaskGenerator.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1taW5pb24tdGFza3MvcGlub3QtbWluaW9uLWJ1aWx0aW4tdGFza3Mvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3BsdWdpbi9taW5pb24vdGFza3MvY29udmVydHRvcmF3aW5kZXgvQ29udmVydFRvUmF3SW5kZXhUYXNrR2VuZXJhdG9yLmphdmE=) | `5.45% <0.00%> (-80.00%)` | :arrow_down: |
   | [...ache/pinot/common/lineage/SegmentLineageUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbGluZWFnZS9TZWdtZW50TGluZWFnZVV0aWxzLmphdmE=) | `22.22% <0.00%> (-77.78%)` | :arrow_down: |
   | [...ore/startree/executor/StarTreeGroupByExecutor.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9zdGFydHJlZS9leGVjdXRvci9TdGFyVHJlZUdyb3VwQnlFeGVjdXRvci5qYXZh) | `0.00% <0.00%> (-77.78%)` | :arrow_down: |
   | ... and [138 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...283b7c3](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (4a01a0f) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `0.31%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     71.06%   70.74%   -0.32%     
   + Complexity     4320     4241      -79     
   ============================================
     Files          1626     1631       +5     
     Lines         85036    85281     +245     
     Branches      12795    12845      +50     
   ============================================
   - Hits          60428    60332      -96     
   - Misses        20456    20781     +325     
   - Partials       4152     4168      +16     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `28.77% <ø> (-0.08%)` | :arrow_down: |
   | integration2 | `27.48% <ø> (-0.06%)` | :arrow_down: |
   | unittests1 | `66.98% <ø> (-0.37%)` | :arrow_down: |
   | unittests2 | `14.15% <ø> (+0.07%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ls/nativefst/automaton/MinimizationOperations.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL01pbmltaXphdGlvbk9wZXJhdGlvbnMuamF2YQ==) | `0.00% <0.00%> (-36.20%)` | :arrow_down: |
   | [...tils/nativefst/automaton/TransitionComparator.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL1RyYW5zaXRpb25Db21wYXJhdG9yLmphdmE=) | `9.37% <0.00%> (-25.00%)` | :arrow_down: |
   | [...ller/helix/core/minion/TaskTypeMetricsUpdater.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL21pbmlvbi9UYXNrVHlwZU1ldHJpY3NVcGRhdGVyLmphdmE=) | `80.00% <0.00%> (-20.00%)` | :arrow_down: |
   | [...ent/local/utils/nativefst/automaton/Automaton.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL0F1dG9tYXRvbi5qYXZh) | `57.08% <0.00%> (-16.86%)` | :arrow_down: |
   | [...gment/spi/partition/HashCodePartitionFunction.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1zcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3NlZ21lbnQvc3BpL3BhcnRpdGlvbi9IYXNoQ29kZVBhcnRpdGlvbkZ1bmN0aW9uLmphdmE=) | `66.66% <0.00%> (-11.12%)` | :arrow_down: |
   | [...cal/utils/nativefst/automaton/BasicOperations.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL0Jhc2ljT3BlcmF0aW9ucy5qYXZh) | `28.57% <0.00%> (-10.85%)` | :arrow_down: |
   | [...pinot/common/utils/fetcher/HttpSegmentFetcher.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvZmV0Y2hlci9IdHRwU2VnbWVudEZldGNoZXIuamF2YQ==) | `61.53% <0.00%> (-10.26%)` | :arrow_down: |
   | [...he/pinot/segment/local/segment/store/IndexKey.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L3N0b3JlL0luZGV4S2V5LmphdmE=) | `65.00% <0.00%> (-10.00%)` | :arrow_down: |
   | [...segment/local/utils/nativefst/automaton/State.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL1N0YXRlLmphdmE=) | `36.58% <0.00%> (-9.76%)` | :arrow_down: |
   | [...nction/DistinctCountBitmapAggregationFunction.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9hZ2dyZWdhdGlvbi9mdW5jdGlvbi9EaXN0aW5jdENvdW50Qml0bWFwQWdncmVnYXRpb25GdW5jdGlvbi5qYXZh) | `47.15% <0.00%> (-8.30%)` | :arrow_down: |
   | ... and [77 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...4a01a0f](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (4a01a0f) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `1.34%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     71.06%   69.71%   -1.35%     
   + Complexity     4320     4241      -79     
   ============================================
     Files          1626     1631       +5     
     Lines         85036    85281     +245     
     Branches      12795    12845      +50     
   ============================================
   - Hits          60428    59454     -974     
   - Misses        20456    21690    +1234     
   + Partials       4152     4137      -15     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `28.77% <ø> (-0.08%)` | :arrow_down: |
   | integration2 | `?` | |
   | unittests1 | `66.98% <ø> (-0.37%)` | :arrow_down: |
   | unittests2 | `14.15% <ø> (+0.07%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...t/core/plan/StreamingInstanceResponsePlanNode.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9wbGFuL1N0cmVhbWluZ0luc3RhbmNlUmVzcG9uc2VQbGFuTm9kZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ore/operator/streaming/StreamingResponseUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9zdHJlYW1pbmcvU3RyZWFtaW5nUmVzcG9uc2VVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ager/realtime/PeerSchemeSplitSegmentCommitter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUGVlclNjaGVtZVNwbGl0U2VnbWVudENvbW1pdHRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/core/plan/StreamingSelectionPlanNode.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9wbGFuL1N0cmVhbWluZ1NlbGVjdGlvblBsYW5Ob2RlLmphdmE=) | `0.00% <0.00%> (-88.89%)` | :arrow_down: |
   | [...ator/streaming/StreamingSelectionOnlyOperator.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9zdHJlYW1pbmcvU3RyZWFtaW5nU2VsZWN0aW9uT25seU9wZXJhdG9yLmphdmE=) | `0.00% <0.00%> (-87.81%)` | :arrow_down: |
   | [...re/query/reduce/SelectionOnlyStreamingReducer.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9yZWR1Y2UvU2VsZWN0aW9uT25seVN0cmVhbWluZ1JlZHVjZXIuamF2YQ==) | `0.00% <0.00%> (-85.72%)` | :arrow_down: |
   | [...pache/pinot/common/utils/grpc/GrpcQueryClient.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvZ3JwYy9HcnBjUXVlcnlDbGllbnQuamF2YQ==) | `0.00% <0.00%> (-80.86%)` | :arrow_down: |
   | [...ller/api/access/BasicAuthAccessControlFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvYWNjZXNzL0Jhc2ljQXV0aEFjY2Vzc0NvbnRyb2xGYWN0b3J5LmphdmE=) | `0.00% <0.00%> (-80.00%)` | :arrow_down: |
   | [...roker/requesthandler/GrpcBrokerRequestHandler.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcmVxdWVzdGhhbmRsZXIvR3JwY0Jyb2tlclJlcXVlc3RIYW5kbGVyLmphdmE=) | `0.00% <0.00%> (-78.58%)` | :arrow_down: |
   | [...re/operator/StreamingInstanceResponseOperator.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9TdHJlYW1pbmdJbnN0YW5jZVJlc3BvbnNlT3BlcmF0b3IuamF2YQ==) | `0.00% <0.00%> (-76.93%)` | :arrow_down: |
   | ... and [147 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...4a01a0f](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (4a01a0f) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `56.91%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #8233       +/-   ##
   =============================================
   - Coverage     71.06%   14.15%   -56.92%     
   + Complexity     4320       81     -4239     
   =============================================
     Files          1626     1586       -40     
     Lines         85036    83399     -1637     
     Branches      12795    12641      -154     
   =============================================
   - Hits          60428    11801    -48627     
   - Misses        20456    70713    +50257     
   + Partials       4152      885     -3267     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `14.15% <ø> (+0.07%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ain/java/org/apache/pinot/core/data/table/Key.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL0tleS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/spi/utils/BooleanUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQm9vbGVhblV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/data/table/Record.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1JlY29yZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/util/GroupByUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dyb3VwQnlVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/config/table/FSTType.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL0ZTVFR5cGUuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/data/MetricFieldSpec.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9NZXRyaWNGaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/spi/utils/BigDecimalUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQmlnRGVjaW1hbFV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/common/tier/TierFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdGllci9UaWVyRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/common/utils/StringUtil.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvU3RyaW5nVXRpbC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/core/data/table/BaseTable.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL0Jhc2VUYWJsZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1306 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...4a01a0f](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (08b2066) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `7.01%`.
   > The diff coverage is `76.99%`.
   
   > :exclamation: Current head 08b2066 differs from pull request most recent head dc67a24. Consider uploading reports for the commit dc67a24 to get more accurate results
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     71.06%   64.04%   -7.02%     
   + Complexity     4320     4243      -77     
   ============================================
     Files          1626     1586      -40     
     Lines         85036    83399    -1637     
     Branches      12795    12641     -154     
   ============================================
   - Hits          60428    53417    -7011     
   - Misses        20456    26140    +5684     
   + Partials       4152     3842     -310     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `66.99% <81.20%> (-0.36%)` | :arrow_down: |
   | unittests2 | `14.10% <17.10%> (+0.02%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...on/src/main/java/org/apache/pinot/serde/SerDe.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zZXJkZS9TZXJEZS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...ler/api/resources/PinotSegmentRestletResource.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL1Bpbm90U2VnbWVudFJlc3RsZXRSZXNvdXJjZS5qYXZh) | `3.38% <0.00%> (-25.13%)` | :arrow_down: |
   | [.../controller/api/resources/PinotTableInstances.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL1Bpbm90VGFibGVJbnN0YW5jZXMuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [.../core/realtime/PinotLLCRealtimeSegmentManager.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL3JlYWx0aW1lL1Bpbm90TExDUmVhbHRpbWVTZWdtZW50TWFuYWdlci5qYXZh) | `66.10% <0.00%> (-11.87%)` | :arrow_down: |
   | [.../org/apache/pinot/core/common/MinionConstants.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9jb21tb24vTWluaW9uQ29uc3RhbnRzLmphdmE=) | `0.00% <ø> (ø)` | |
   | [...manager/realtime/LLRealtimeSegmentDataManager.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvTExSZWFsdGltZVNlZ21lbnREYXRhTWFuYWdlci5qYXZh) | `45.44% <0.00%> (-25.40%)` | :arrow_down: |
   | [...ery/optimizer/filter/NumericalFilterOptimizer.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9vcHRpbWl6ZXIvZmlsdGVyL051bWVyaWNhbEZpbHRlck9wdGltaXplci5qYXZh) | `83.22% <ø> (-2.49%)` | :arrow_down: |
   | [.../org/apache/pinot/plugin/filesystem/S3PinotFS.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1maWxlLXN5c3RlbS9waW5vdC1zMy9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcGx1Z2luL2ZpbGVzeXN0ZW0vUzNQaW5vdEZTLmphdmE=) | `38.70% <0.00%> (ø)` | |
   | [...ot/plugin/minion/tasks/SegmentConversionUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1taW5pb24tdGFza3MvcGlub3QtbWluaW9uLWJ1aWx0aW4tdGFza3Mvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3BsdWdpbi9taW5pb24vdGFza3MvU2VnbWVudENvbnZlcnNpb25VdGlscy5qYXZh) | `0.00% <0.00%> (-76.28%)` | :arrow_down: |
   | [...ocal/indexsegment/mutable/IntermediateSegment.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvbXV0YWJsZS9JbnRlcm1lZGlhdGVTZWdtZW50LmphdmE=) | `69.62% <0.00%> (-0.45%)` | :arrow_down: |
   | ... and [445 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...dc67a24](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r819252986



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,266 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a buffer. The {@link GenericRow} are written to
+ * the buffer as AVRO records.
+ */
+@SuppressWarnings("NullAway")
+@NotThreadSafe
+public class FlinkSegmentWriter implements SegmentWriter {

Review comment:
       That's fair. Let me address this in a follow-up PR, and it shouldn't be hard. But I try not to modify other modules in this PR, so it'll be a pure module addition.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] xiangfu0 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r812487512



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/PinotControllerClient.java
##########
@@ -0,0 +1,147 @@
+/**
+ * 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.connector.flink.http;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonValue;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/** This file is partially copied from RTAUMS gateway package */
+public class PinotControllerClient extends HttpClient {

Review comment:
       I feel this is a very helpful Client. 
   
   Just drop my random thought here:
   Shall we rename this to PinotAdminClient and move it to the `pinot-java-client` module then import it back?
   
   cc: @KKcorps @Jackie-Jiang 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (657daa5) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `1.59%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     71.06%   69.46%   -1.60%     
   + Complexity     4320     4240      -80     
   ============================================
     Files          1626     1631       +5     
     Lines         85036    85281     +245     
     Branches      12795    12845      +50     
   ============================================
   - Hits          60428    59243    -1185     
   - Misses        20456    21905    +1449     
   + Partials       4152     4133      -19     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `27.52% <ø> (-0.02%)` | :arrow_down: |
   | unittests1 | `66.96% <ø> (-0.39%)` | :arrow_down: |
   | unittests2 | `14.10% <ø> (+0.03%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...pinot/minion/exception/TaskCancelledException.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vZXhjZXB0aW9uL1Rhc2tDYW5jZWxsZWRFeGNlcHRpb24uamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...nverttorawindex/ConvertToRawIndexTaskExecutor.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1taW5pb24tdGFza3MvcGlub3QtbWluaW9uLWJ1aWx0aW4tdGFza3Mvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3BsdWdpbi9taW5pb24vdGFza3MvY29udmVydHRvcmF3aW5kZXgvQ29udmVydFRvUmF3SW5kZXhUYXNrRXhlY3V0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...e/pinot/common/minion/MergeRollupTaskMetadata.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWluaW9uL01lcmdlUm9sbHVwVGFza01ldGFkYXRhLmphdmE=) | `0.00% <0.00%> (-94.74%)` | :arrow_down: |
   | [...plugin/segmentuploader/SegmentUploaderDefault.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1zZWdtZW50LXVwbG9hZGVyL3Bpbm90LXNlZ21lbnQtdXBsb2FkZXItZGVmYXVsdC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcGx1Z2luL3NlZ21lbnR1cGxvYWRlci9TZWdtZW50VXBsb2FkZXJEZWZhdWx0LmphdmE=) | `0.00% <0.00%> (-87.10%)` | :arrow_down: |
   | [.../transform/function/MapValueTransformFunction.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vTWFwVmFsdWVUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `0.00% <0.00%> (-85.30%)` | :arrow_down: |
   | [...ot/common/messages/RoutingTableRebuildMessage.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvUm91dGluZ1RhYmxlUmVidWlsZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-81.82%)` | :arrow_down: |
   | [...data/manager/realtime/DefaultSegmentCommitter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvRGVmYXVsdFNlZ21lbnRDb21taXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-80.00%)` | :arrow_down: |
   | [...verttorawindex/ConvertToRawIndexTaskGenerator.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1taW5pb24tdGFza3MvcGlub3QtbWluaW9uLWJ1aWx0aW4tdGFza3Mvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3BsdWdpbi9taW5pb24vdGFza3MvY29udmVydHRvcmF3aW5kZXgvQ29udmVydFRvUmF3SW5kZXhUYXNrR2VuZXJhdG9yLmphdmE=) | `5.45% <0.00%> (-80.00%)` | :arrow_down: |
   | [...ache/pinot/common/lineage/SegmentLineageUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbGluZWFnZS9TZWdtZW50TGluZWFnZVV0aWxzLmphdmE=) | `22.22% <0.00%> (-77.78%)` | :arrow_down: |
   | [...ore/startree/executor/StarTreeGroupByExecutor.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9zdGFydHJlZS9leGVjdXRvci9TdGFyVHJlZUdyb3VwQnlFeGVjdXRvci5qYXZh) | `0.00% <0.00%> (-77.78%)` | :arrow_down: |
   | ... and [166 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...657daa5](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] walterddr commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
walterddr commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r815384092



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/HttpClient.java
##########
@@ -0,0 +1,130 @@
+/**

Review comment:
       hmm. I must have missed something. here is what I did:
   1. deleted the entire folder under `src/main/java/org/apache/pinot/connector/flink/http`.
   2. run `mvn test -pl pinot-connectors/pinot-flink-connector`
   
   everything runs pass. this proves that the http folder has nothing to do with this PR. yes?
   
   did you forget to add a Quickstart? I don't see a quickstart in this PR or any changes outside of the pinot-flink-connector module. (and if so, let's add it in a separate PR i suggest)




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (08b2066) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `56.95%`.
   > The diff coverage is `17.10%`.
   
   > :exclamation: Current head 08b2066 differs from pull request most recent head dc67a24. Consider uploading reports for the commit dc67a24 to get more accurate results
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #8233       +/-   ##
   =============================================
   - Coverage     71.06%   14.10%   -56.96%     
   + Complexity     4320       81     -4239     
   =============================================
     Files          1626     1586       -40     
     Lines         85036    83399     -1637     
     Branches      12795    12641      -154     
   =============================================
   - Hits          60428    11761    -48667     
   - Misses        20456    70759    +50303     
   + Partials       4152      879     -3273     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `14.10% <17.10%> (+0.02%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...apache/pinot/common/function/FunctionRegistry.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25SZWdpc3RyeS5qYXZh) | `0.00% <0.00%> (-87.10%)` | :arrow_down: |
   | [...he/pinot/common/request/context/FilterContext.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vcmVxdWVzdC9jb250ZXh0L0ZpbHRlckNvbnRleHQuamF2YQ==) | `0.00% <0.00%> (-78.38%)` | :arrow_down: |
   | [...ot/common/request/context/RequestContextUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vcmVxdWVzdC9jb250ZXh0L1JlcXVlc3RDb250ZXh0VXRpbHMuamF2YQ==) | `0.00% <0.00%> (-74.38%)` | :arrow_down: |
   | [.../apache/pinot/pql/parsers/pql2/ast/FilterKind.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wcWwvcGFyc2Vycy9wcWwyL2FzdC9GaWx0ZXJLaW5kLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...on/src/main/java/org/apache/pinot/serde/SerDe.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zZXJkZS9TZXJEZS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...org/apache/pinot/sql/parsers/CalciteSqlParser.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcWwvcGFyc2Vycy9DYWxjaXRlU3FsUGFyc2VyLmphdmE=) | `0.00% <0.00%> (-87.93%)` | :arrow_down: |
   | [...ler/api/resources/PinotSegmentRestletResource.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL1Bpbm90U2VnbWVudFJlc3RsZXRSZXNvdXJjZS5qYXZh) | `3.38% <0.00%> (-25.13%)` | :arrow_down: |
   | [.../controller/api/resources/PinotTableInstances.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL1Bpbm90VGFibGVJbnN0YW5jZXMuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [.../core/realtime/PinotLLCRealtimeSegmentManager.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL3JlYWx0aW1lL1Bpbm90TExDUmVhbHRpbWVTZWdtZW50TWFuYWdlci5qYXZh) | `66.10% <0.00%> (-11.87%)` | :arrow_down: |
   | [.../org/apache/pinot/core/common/MinionConstants.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9jb21tb24vTWluaW9uQ29uc3RhbnRzLmphdmE=) | `0.00% <ø> (ø)` | |
   | ... and [1359 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...dc67a24](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] npawar commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
npawar commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r820306620



##########
File path: pinot-connectors/pinot-flink-connector/src/test/java/org/apache/pinot/connector/flink/util/TestUtils.java
##########
@@ -0,0 +1,44 @@
+/**
+ * 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.connector.flink.util;
+
+import java.io.File;
+import java.io.IOException;
+import javax.annotation.Nonnull;
+import org.apache.commons.io.FileUtils;
+import org.testng.Assert;
+
+
+public final class TestUtils {

Review comment:
       it is fine to add the test dependency for connectors to pinot-integration-tests.. we add dependency of required plugin/connector there anyway for the test. fine if this is addressed in next PR




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] npawar commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
npawar commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r820306739



##########
File path: pinot-connectors/pinot-flink-connector/src/test/resources/fixtures/pinotTableSchema.json
##########
@@ -0,0 +1,45 @@
+{
+  "schemaName": "demand",
+  "dimensionFieldSpecs": [
+    {
+      "name": "demand_uuid",
+      "singleValueField": true,
+      "dataType": "STRING",
+      "virtualColumnProvider": null,
+      "defaultNullValue": "null",
+      "transformFunction": null,
+      "maxLength" : 4096
+    },
+    {
+      "name": "geofence",
+      "singleValueField": true,
+      "dataType": "STRING",
+      "virtualColumnProvider": null,
+      "defaultNullValue": "null",
+      "transformFunction": null
+    }
+  ],
+  "metricFieldSpecs": [
+    {
+      "dataType": "DOUBLE",
+      "singleValueField": true,
+      "fieldSize": 8,
+      "derivedMetricType": null,
+      "name": "surge_multiplier",
+      "virtualColumnProvider": null,
+      "defaultNullValue": 0.0,
+      "transformFunction": null
+    }
+  ],
+  "dateTimeFieldSpecs": [

Review comment:
       how is this not failing with 2 `dateTimeFieldSpecs` !




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (283b7c3) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `0.31%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     71.06%   70.74%   -0.32%     
   + Complexity     4320     4241      -79     
   ============================================
     Files          1626     1629       +3     
     Lines         85036    85152     +116     
     Branches      12795    12815      +20     
   ============================================
   - Hits          60428    60241     -187     
   - Misses        20456    20754     +298     
   - Partials       4152     4157       +5     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `28.83% <ø> (-0.03%)` | :arrow_down: |
   | integration2 | `27.49% <ø> (-0.05%)` | :arrow_down: |
   | unittests1 | `66.96% <ø> (-0.39%)` | :arrow_down: |
   | unittests2 | `14.11% <ø> (+0.03%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ls/nativefst/automaton/MinimizationOperations.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL01pbmltaXphdGlvbk9wZXJhdGlvbnMuamF2YQ==) | `0.00% <0.00%> (-36.20%)` | :arrow_down: |
   | [...tils/nativefst/automaton/TransitionComparator.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL1RyYW5zaXRpb25Db21wYXJhdG9yLmphdmE=) | `9.37% <0.00%> (-25.00%)` | :arrow_down: |
   | [...ller/helix/core/minion/TaskTypeMetricsUpdater.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL21pbmlvbi9UYXNrVHlwZU1ldHJpY3NVcGRhdGVyLmphdmE=) | `80.00% <0.00%> (-20.00%)` | :arrow_down: |
   | [...ent/local/utils/nativefst/automaton/Automaton.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL0F1dG9tYXRvbi5qYXZh) | `57.08% <0.00%> (-16.86%)` | :arrow_down: |
   | [...gment/spi/partition/HashCodePartitionFunction.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1zcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3NlZ21lbnQvc3BpL3BhcnRpdGlvbi9IYXNoQ29kZVBhcnRpdGlvbkZ1bmN0aW9uLmphdmE=) | `66.66% <0.00%> (-11.12%)` | :arrow_down: |
   | [...cal/utils/nativefst/automaton/BasicOperations.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL0Jhc2ljT3BlcmF0aW9ucy5qYXZh) | `28.57% <0.00%> (-10.85%)` | :arrow_down: |
   | [...pinot/common/utils/fetcher/HttpSegmentFetcher.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvZmV0Y2hlci9IdHRwU2VnbWVudEZldGNoZXIuamF2YQ==) | `61.53% <0.00%> (-10.26%)` | :arrow_down: |
   | [...segment/local/utils/nativefst/automaton/State.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL1N0YXRlLmphdmE=) | `36.58% <0.00%> (-9.76%)` | :arrow_down: |
   | [...he/pinot/segment/local/segment/store/IndexKey.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L3N0b3JlL0luZGV4S2V5LmphdmE=) | `70.00% <0.00%> (-5.00%)` | :arrow_down: |
   | [.../plugin/inputformat/thrift/ThriftRecordReader.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtdGhyaWZ0L3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvdGhyaWZ0L1RocmlmdFJlY29yZFJlYWRlci5qYXZh) | `86.95% <0.00%> (-3.75%)` | :arrow_down: |
   | ... and [53 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...283b7c3](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (dff84c0) into [master](https://codecov.io/gh/apache/pinot/commit/8f0e49ed804f63bd583ac207a41808e3b059b82f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f0e49e) will **decrease** coverage by `0.38%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8233      +/-   ##
   ============================================
   - Coverage     71.06%   70.67%   -0.39%     
   + Complexity     4320     4239      -81     
   ============================================
     Files          1626     1629       +3     
     Lines         85036    85212     +176     
     Branches      12795    12830      +35     
   ============================================
   - Hits          60428    60226     -202     
   - Misses        20456    20819     +363     
   - Partials       4152     4167      +15     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `28.71% <ø> (-0.15%)` | :arrow_down: |
   | integration2 | `27.58% <ø> (+0.04%)` | :arrow_up: |
   | unittests1 | `66.94% <ø> (-0.41%)` | :arrow_down: |
   | unittests2 | `14.12% <ø> (+0.04%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...data/manager/realtime/DefaultSegmentCommitter.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvRGVmYXVsdFNlZ21lbnRDb21taXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-80.00%)` | :arrow_down: |
   | [...ls/nativefst/automaton/MinimizationOperations.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL01pbmltaXphdGlvbk9wZXJhdGlvbnMuamF2YQ==) | `0.00% <0.00%> (-36.20%)` | :arrow_down: |
   | [...tils/nativefst/automaton/TransitionComparator.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL1RyYW5zaXRpb25Db21wYXJhdG9yLmphdmE=) | `9.37% <0.00%> (-25.00%)` | :arrow_down: |
   | [...ller/helix/core/minion/TaskTypeMetricsUpdater.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL21pbmlvbi9UYXNrVHlwZU1ldHJpY3NVcGRhdGVyLmphdmE=) | `80.00% <0.00%> (-20.00%)` | :arrow_down: |
   | [...er/api/resources/LLCSegmentCompletionHandlers.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL0xMQ1NlZ21lbnRDb21wbGV0aW9uSGFuZGxlcnMuamF2YQ==) | `43.56% <0.00%> (-18.82%)` | :arrow_down: |
   | [...ent/local/utils/nativefst/automaton/Automaton.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL0F1dG9tYXRvbi5qYXZh) | `57.08% <0.00%> (-16.86%)` | :arrow_down: |
   | [...data/manager/realtime/SegmentCommitterFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvU2VnbWVudENvbW1pdHRlckZhY3RvcnkuamF2YQ==) | `88.23% <0.00%> (-11.77%)` | :arrow_down: |
   | [...gment/spi/partition/HashCodePartitionFunction.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1zcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3NlZ21lbnQvc3BpL3BhcnRpdGlvbi9IYXNoQ29kZVBhcnRpdGlvbkZ1bmN0aW9uLmphdmE=) | `66.66% <0.00%> (-11.12%)` | :arrow_down: |
   | [...cal/utils/nativefst/automaton/BasicOperations.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9uYXRpdmVmc3QvYXV0b21hdG9uL0Jhc2ljT3BlcmF0aW9ucy5qYXZh) | `28.57% <0.00%> (-10.85%)` | :arrow_down: |
   | [...pinot/common/utils/fetcher/HttpSegmentFetcher.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvZmV0Y2hlci9IdHRwU2VnbWVudEZldGNoZXIuamF2YQ==) | `61.53% <0.00%> (-10.26%)` | :arrow_down: |
   | ... and [58 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f0e49e...dff84c0](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r818999292



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,266 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a buffer. The {@link GenericRow} are written to
+ * the buffer as AVRO records.
+ */
+@SuppressWarnings("NullAway")
+@NotThreadSafe
+public class FlinkSegmentWriter implements SegmentWriter {

Review comment:
       We could with a later refactoring. I think one of the reasons is that we don't have an abstract class of `SegmentWriter` that allows customization like in this Flink case.  Also, initially, this connector was not planned to add to Pinot repo, but Flink repo, which means this connector has to depend on the published pinot library only. 
   
   Similar to the consolidation with the client utils, I think we can later consolidate the SegmentWriter, after they are in the same repo.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] npawar commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
npawar commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r819260880



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,266 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a buffer. The {@link GenericRow} are written to
+ * the buffer as AVRO records.
+ */
+@SuppressWarnings("NullAway")
+@NotThreadSafe
+public class FlinkSegmentWriter implements SegmentWriter {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FlinkSegmentWriter.class);
+  private static final FileFormat BUFFER_FILE_FORMAT = FileFormat.AVRO;
+
+  private final int _indexOfSubtask;
+
+  private TableConfig _tableConfig;
+  private String _tableNameWithType;
+  private BatchIngestionConfig _batchIngestionConfig;
+  private BatchConfig _batchConfig;
+  private String _outputDirURI;
+  private Schema _schema;
+  private Set<String> _fieldsToRead;
+  private RecordTransformer _recordTransformer;
+
+  private File _stagingDir;
+  private File _bufferFile;
+  private int _rowCount;
+  /** A sequence ID that increments each time a segment is flushed */
+  private int _seqId;
+
+  private org.apache.avro.Schema _avroSchema;
+  private DataFileWriter<GenericData.Record> _recordWriter;
+  private GenericData.Record _reusableRecord;
+
+  // metrics
+  private transient Counter _processedRecords;
+  private transient volatile long _lastRecordProcessingTimeMs = 0;
+
+  public FlinkSegmentWriter(int indexOfSubtask, MetricGroup metricGrp) {
+    _indexOfSubtask = indexOfSubtask;
+    registerMetrics(metricGrp);
+  }
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema)
+      throws Exception {
+    init(tableConfig, schema, null);
+  }
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema, Map<String, String> batchConfigOverride)
+      throws Exception {
+    _rowCount = 0;
+    _seqId = 0;
+    _tableConfig = tableConfig;
+    _tableNameWithType = _tableConfig.getTableName();
+    Preconditions.checkState(
+        _tableConfig.getIngestionConfig() != null && _tableConfig.getIngestionConfig().getBatchIngestionConfig() != null
+            && CollectionUtils.isNotEmpty(
+            _tableConfig.getIngestionConfig().getBatchIngestionConfig().getBatchConfigMaps()),
+        "Must provide ingestionConfig->batchIngestionConfig->batchConfigMaps in tableConfig for table: %s",
+        _tableNameWithType);
+    _batchIngestionConfig = _tableConfig.getIngestionConfig().getBatchIngestionConfig();
+    Preconditions.checkState(_batchIngestionConfig.getBatchConfigMaps().size() == 1,
+        "batchConfigMaps must contain only 1 BatchConfig for table: %s", _tableNameWithType);
+
+    Map<String, String> batchConfigMap = _batchIngestionConfig.getBatchConfigMaps().get(0);
+    String segmentNamePostfixProp = String.format("%s.%s", BatchConfigProperties.SEGMENT_NAME_GENERATOR_PROP_PREFIX,
+        BatchConfigProperties.SEGMENT_NAME_POSTFIX);
+    String segmentSuffix = batchConfigMap.get(segmentNamePostfixProp);
+    segmentSuffix = segmentSuffix == null ? String.valueOf(_indexOfSubtask) : segmentSuffix + "_" + _indexOfSubtask;
+    batchConfigMap.put(segmentNamePostfixProp, segmentSuffix);
+
+    _batchConfig = new BatchConfig(_tableNameWithType, batchConfigMap);
+
+    Preconditions.checkState(StringUtils.isNotBlank(_batchConfig.getOutputDirURI()),
+        "Must provide: %s in batchConfigs for table: %s", BatchConfigProperties.OUTPUT_DIR_URI, _tableNameWithType);
+    _outputDirURI = _batchConfig.getOutputDirURI();
+    Files.createDirectories(Paths.get(_outputDirURI));
+
+    _schema = schema;
+    _fieldsToRead = _schema.getColumnNames();
+    _recordTransformer = CompositeTransformer.getDefaultTransformer(_tableConfig, _schema);
+    _avroSchema = SegmentProcessorAvroUtils.convertPinotSchemaToAvroSchema(_schema);
+    _reusableRecord = new GenericData.Record(_avroSchema);
+
+    // Create tmp dir
+    _stagingDir = new File(FileUtils.getTempDirectory(),
+        String.format("segment_writer_staging_%s_%d_%d", _tableNameWithType, _indexOfSubtask,
+            System.currentTimeMillis()));
+    Preconditions.checkState(_stagingDir.mkdirs(), "Failed to create staging dir: %s", _stagingDir.getAbsolutePath());
+
+    // Create buffer file
+    File bufferDir = new File(_stagingDir, "buffer_dir");
+    Preconditions.checkState(bufferDir.mkdirs(), "Failed to create buffer_dir: %s", bufferDir.getAbsolutePath());
+    _bufferFile = new File(bufferDir, "buffer_file");
+    resetBuffer();
+    LOGGER.info("Initialized {} for Pinot table: {}", this.getClass().getName(), _tableNameWithType);
+  }
+
+  private void registerMetrics(MetricGroup metricGrp) {
+    _processedRecords = metricGrp.counter("records.processed");
+    metricGrp.gauge("record.processing.time.ts", (Gauge<Long>) () -> _lastRecordProcessingTimeMs);
+  }
+
+  private void resetBuffer()
+      throws IOException {
+    FileUtils.deleteQuietly(_bufferFile);
+    _rowCount = 0;
+    _recordWriter = new DataFileWriter<>(new GenericDatumWriter<>(_avroSchema));
+    _recordWriter.create(_avroSchema, _bufferFile);
+  }
+
+  @Override
+  public void collect(GenericRow row)
+      throws IOException {
+    long startTime = System.currentTimeMillis();
+    GenericRow transform = _recordTransformer.transform(row);
+    SegmentProcessorAvroUtils.convertGenericRowToAvroRecord(transform, _reusableRecord, _fieldsToRead);
+    _rowCount++;
+    _recordWriter.append(_reusableRecord);
+    _lastRecordProcessingTimeMs = System.currentTimeMillis() - startTime;
+    _processedRecords.inc();
+  }
+
+  /**
+   * Creates one Pinot segment using the {@link GenericRow}s collected in the AVRO file buffer, at
+   * the outputDirUri as specified in the tableConfig->batchConfigs. Successful invocation of this
+   * method means that the {@link GenericRow}s collected so far, are now available in the Pinot
+   * segment and not available in the buffer anymore.
+   *
+   * <p>Successful completion of segment will return the segment URI, and the URI includes a
+   * sequence id indicating the part number. The sequence id is initialized to 0 and each successful
+   * flush will increment the sequence id by 1. The buffer will be reset and ready to accept further
+   * records via <code>collect()</code> If an exception is thrown, the buffer will not be reset and
+   * so, <code>flush()</code> can be invoked repeatedly in a retry loop. If a successful invocation
+   * is not achieved,<code>close()</code> followed by <code>init </code> will have to be called in
+   * order to reset the buffer and resume record writing.
+   *
+   * @return URI of the generated segment
+   * @throws IOException
+   */
+  @Override
+  public URI flush()
+      throws IOException {
+
+    LOGGER.info("Beginning flush for Pinot table: {} with {} records", _tableNameWithType, _rowCount);
+    _recordWriter.close();
+
+    // Create temp dir for flush
+    File flushDir = new File(_stagingDir, "flush_dir_" + System.currentTimeMillis());
+    Preconditions.checkState(flushDir.mkdirs(), "Failed to create flush dir: %s", flushDir);
+
+    try {
+      // Segment dir
+      File segmentDir = new File(flushDir, "segment_dir");
+
+      // Make BatchIngestionConfig for flush
+      Map<String, String> batchConfigMapOverride = new HashMap<>(_batchConfig.getBatchConfigMap());
+      batchConfigMapOverride.put(BatchConfigProperties.INPUT_DIR_URI, _bufferFile.getAbsolutePath());
+      batchConfigMapOverride.put(BatchConfigProperties.OUTPUT_DIR_URI, segmentDir.getAbsolutePath());
+      batchConfigMapOverride.put(BatchConfigProperties.INPUT_FORMAT, BUFFER_FILE_FORMAT.toString());
+      BatchIngestionConfig batchIngestionConfig = new BatchIngestionConfig(Lists.newArrayList(batchConfigMapOverride),
+          _batchIngestionConfig.getSegmentIngestionType(), _batchIngestionConfig.getSegmentIngestionFrequency());
+
+      // Build segment
+      SegmentGeneratorConfig segmentGeneratorConfig =
+          IngestionUtils.generateSegmentGeneratorConfig(_tableConfig, _schema, batchIngestionConfig);
+      segmentGeneratorConfig.setSequenceId(_seqId);
+      String segmentName = IngestionUtils.buildSegment(segmentGeneratorConfig);
+      LOGGER.info("Successfully built segment: {} of sequence {} for Pinot table: {}", segmentName, _seqId,
+          _tableNameWithType);
+
+      // Tar segment
+      File segmentTarFile =
+          new File(_outputDirURI, String.format("%s_%d%s", segmentName, _indexOfSubtask, Constants.TAR_GZ_FILE_EXT));

Review comment:
       yea, indexOfSubtask is added to postfix above
   ```
   String segmentNamePostfixProp = String.format("%s.%s", BatchConfigProperties.SEGMENT_NAME_GENERATOR_PROP_PREFIX,
           BatchConfigProperties.SEGMENT_NAME_POSTFIX);
       String segmentSuffix = batchConfigMap.get(segmentNamePostfixProp);
       segmentSuffix = segmentSuffix == null ? String.valueOf(_indexOfSubtask) : segmentSuffix + "_" + _indexOfSubtask;
       batchConfigMap.put(segmentNamePostfixProp, segmentSuffix);
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 commented on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046284610


   @walterddr please review


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r815387819



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/HttpClient.java
##########
@@ -0,0 +1,130 @@
+/**

Review comment:
       hmm , I see. This file is filtered by one of the .gitignore rules. Renamed it and pushed again.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] walterddr commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
walterddr commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r815384092



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/HttpClient.java
##########
@@ -0,0 +1,130 @@
+/**

Review comment:
       hmm. I must have missed something. here is what I did:
   1. deleted the entire folder under `src/main/java/org/apache/pinot/connector/flink/http`.
   2. run `mvn test -pl pinot-connectors/pinot-flink-connector`
   
   everything runs pass. 
   
   did you forget to add a Quickstart? I don't see a quickstart in this PR or any changes outside of the pinot-flink-connector module. 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] walterddr commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
walterddr commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r815377380



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/HttpClient.java
##########
@@ -0,0 +1,130 @@
+/**

Review comment:
       yes, but it is not related to the flink connector. for the sake of this PR's scope. i would suggest we drop it and add in separate PR.
   
   mainly for the following reasons. 
   - it doesn't relate to flink connector, at first I thought we are utilizing http to directly ingest into pinot via some sort of REST API. but if it is not the case it shouldn't be part of flink-pinot connector but rather in pinot-tools. 
   - @xiangfu0 also think we can refactor this part out later. doesn't make sense to get this in an refactor it out if it is not part of the critical path
   - there's already one util in test call ControllerUtils that can be used for demonstration purpose.
   
   Not saying it is not useful, but it would be much easier to address it separately. 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r820375244



##########
File path: pinot-connectors/pinot-flink-connector/src/test/resources/fixtures/pinotTableSchema.json
##########
@@ -0,0 +1,45 @@
+{
+  "schemaName": "demand",
+  "dimensionFieldSpecs": [
+    {
+      "name": "demand_uuid",
+      "singleValueField": true,
+      "dataType": "STRING",
+      "virtualColumnProvider": null,
+      "defaultNullValue": "null",
+      "transformFunction": null,
+      "maxLength" : 4096
+    },
+    {
+      "name": "geofence",
+      "singleValueField": true,
+      "dataType": "STRING",
+      "virtualColumnProvider": null,
+      "defaultNullValue": "null",
+      "transformFunction": null
+    }
+  ],
+  "metricFieldSpecs": [
+    {
+      "dataType": "DOUBLE",
+      "singleValueField": true,
+      "fieldSize": 8,
+      "derivedMetricType": null,
+      "name": "surge_multiplier",
+      "virtualColumnProvider": null,
+      "defaultNullValue": 0.0,
+      "transformFunction": null
+    }
+  ],
+  "dateTimeFieldSpecs": [

Review comment:
       oh, it's just a dummy file to test the RPC calls




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] npawar commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
npawar commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r818236177



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,266 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a buffer. The {@link GenericRow} are written to
+ * the buffer as AVRO records.
+ */
+@SuppressWarnings("NullAway")
+@NotThreadSafe
+public class FlinkSegmentWriter implements SegmentWriter {

Review comment:
       why was creating a Flink specific impl of SegmentWriter needed? Why couldn't we just use the FileBasedSegmentWriter? most of the code looks the same, with some additional metadata like seqId and rowCount etc. Aren't those concepts generic enough that we could just add them to FileBasedSegmentWriter and enhance it?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#issuecomment-1046289198


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8233](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (05d2c8e) into [master](https://codecov.io/gh/apache/pinot/commit/46ed731c4e60c308c9559e46349a984b0ce05ce6?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (46ed731) will **decrease** coverage by `56.66%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pinot/pull/8233/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #8233       +/-   ##
   =============================================
   - Coverage     70.83%   14.17%   -56.67%     
   + Complexity     4245       81     -4164     
   =============================================
     Files          1631     1586       -45     
     Lines         85462    83580     -1882     
     Branches      12877    12673      -204     
   =============================================
   - Hits          60539    11849    -48690     
   - Misses        20746    70848    +50102     
   + Partials       4177      883     -3294     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `14.17% <ø> (+0.01%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ain/java/org/apache/pinot/core/data/table/Key.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL0tleS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/spi/utils/BooleanUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQm9vbGVhblV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/data/table/Record.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1JlY29yZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/util/GroupByUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dyb3VwQnlVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/config/table/FSTType.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL0ZTVFR5cGUuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/data/MetricFieldSpec.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9NZXRyaWNGaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/spi/utils/BigDecimalUtils.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQmlnRGVjaW1hbFV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/common/tier/TierFactory.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdGllci9UaWVyRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/common/utils/StringUtil.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvU3RyaW5nVXRpbC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/core/data/table/BaseTable.java](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL0Jhc2VUYWJsZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1299 more](https://codecov.io/gh/apache/pinot/pull/8233/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [46ed731...05d2c8e](https://codecov.io/gh/apache/pinot/pull/8233?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] npawar commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
npawar commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r819011364



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,266 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a buffer. The {@link GenericRow} are written to
+ * the buffer as AVRO records.
+ */
+@SuppressWarnings("NullAway")
+@NotThreadSafe
+public class FlinkSegmentWriter implements SegmentWriter {

Review comment:
       the Sink should have been in Flink, but the Writer should have always been in Pinot, and it should have been from the impls offered. If by consolidate later, you mean soon enough, I'm fine with starting with this. But this can easily get deprioritized and fall off the plate once it is non-blocking :)  And the code really is identical, except a few tweaks here and there which the default impl can benefit from anyway..




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] yupeng9 commented on a change in pull request #8233: Add Flink Pinot connector

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #8233:
URL: https://github.com/apache/pinot/pull/8233#discussion_r819700662



##########
File path: pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/FlinkSegmentWriter.java
##########
@@ -0,0 +1,266 @@
+/**
+ * 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.connector.flink.sink;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer;
+import org.apache.pinot.segment.local.recordtransformer.RecordTransformer;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a buffer. The {@link GenericRow} are written to
+ * the buffer as AVRO records.
+ */
+@SuppressWarnings("NullAway")
+@NotThreadSafe
+public class FlinkSegmentWriter implements SegmentWriter {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FlinkSegmentWriter.class);
+  private static final FileFormat BUFFER_FILE_FORMAT = FileFormat.AVRO;
+
+  private final int _indexOfSubtask;
+
+  private TableConfig _tableConfig;
+  private String _tableNameWithType;
+  private BatchIngestionConfig _batchIngestionConfig;
+  private BatchConfig _batchConfig;
+  private String _outputDirURI;
+  private Schema _schema;
+  private Set<String> _fieldsToRead;
+  private RecordTransformer _recordTransformer;
+
+  private File _stagingDir;
+  private File _bufferFile;
+  private int _rowCount;
+  /** A sequence ID that increments each time a segment is flushed */
+  private int _seqId;
+
+  private org.apache.avro.Schema _avroSchema;
+  private DataFileWriter<GenericData.Record> _recordWriter;
+  private GenericData.Record _reusableRecord;
+
+  // metrics
+  private transient Counter _processedRecords;
+  private transient volatile long _lastRecordProcessingTimeMs = 0;
+
+  public FlinkSegmentWriter(int indexOfSubtask, MetricGroup metricGrp) {
+    _indexOfSubtask = indexOfSubtask;
+    registerMetrics(metricGrp);
+  }
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema)
+      throws Exception {
+    init(tableConfig, schema, null);
+  }
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema, Map<String, String> batchConfigOverride)
+      throws Exception {
+    _rowCount = 0;
+    _seqId = 0;
+    _tableConfig = tableConfig;
+    _tableNameWithType = _tableConfig.getTableName();
+    Preconditions.checkState(
+        _tableConfig.getIngestionConfig() != null && _tableConfig.getIngestionConfig().getBatchIngestionConfig() != null
+            && CollectionUtils.isNotEmpty(
+            _tableConfig.getIngestionConfig().getBatchIngestionConfig().getBatchConfigMaps()),
+        "Must provide ingestionConfig->batchIngestionConfig->batchConfigMaps in tableConfig for table: %s",
+        _tableNameWithType);
+    _batchIngestionConfig = _tableConfig.getIngestionConfig().getBatchIngestionConfig();
+    Preconditions.checkState(_batchIngestionConfig.getBatchConfigMaps().size() == 1,
+        "batchConfigMaps must contain only 1 BatchConfig for table: %s", _tableNameWithType);
+
+    Map<String, String> batchConfigMap = _batchIngestionConfig.getBatchConfigMaps().get(0);
+    String segmentNamePostfixProp = String.format("%s.%s", BatchConfigProperties.SEGMENT_NAME_GENERATOR_PROP_PREFIX,
+        BatchConfigProperties.SEGMENT_NAME_POSTFIX);
+    String segmentSuffix = batchConfigMap.get(segmentNamePostfixProp);
+    segmentSuffix = segmentSuffix == null ? String.valueOf(_indexOfSubtask) : segmentSuffix + "_" + _indexOfSubtask;
+    batchConfigMap.put(segmentNamePostfixProp, segmentSuffix);
+
+    _batchConfig = new BatchConfig(_tableNameWithType, batchConfigMap);
+
+    Preconditions.checkState(StringUtils.isNotBlank(_batchConfig.getOutputDirURI()),
+        "Must provide: %s in batchConfigs for table: %s", BatchConfigProperties.OUTPUT_DIR_URI, _tableNameWithType);
+    _outputDirURI = _batchConfig.getOutputDirURI();
+    Files.createDirectories(Paths.get(_outputDirURI));
+
+    _schema = schema;
+    _fieldsToRead = _schema.getColumnNames();
+    _recordTransformer = CompositeTransformer.getDefaultTransformer(_tableConfig, _schema);
+    _avroSchema = SegmentProcessorAvroUtils.convertPinotSchemaToAvroSchema(_schema);
+    _reusableRecord = new GenericData.Record(_avroSchema);
+
+    // Create tmp dir
+    _stagingDir = new File(FileUtils.getTempDirectory(),
+        String.format("segment_writer_staging_%s_%d_%d", _tableNameWithType, _indexOfSubtask,
+            System.currentTimeMillis()));
+    Preconditions.checkState(_stagingDir.mkdirs(), "Failed to create staging dir: %s", _stagingDir.getAbsolutePath());
+
+    // Create buffer file
+    File bufferDir = new File(_stagingDir, "buffer_dir");
+    Preconditions.checkState(bufferDir.mkdirs(), "Failed to create buffer_dir: %s", bufferDir.getAbsolutePath());
+    _bufferFile = new File(bufferDir, "buffer_file");
+    resetBuffer();
+    LOGGER.info("Initialized {} for Pinot table: {}", this.getClass().getName(), _tableNameWithType);
+  }
+
+  private void registerMetrics(MetricGroup metricGrp) {
+    _processedRecords = metricGrp.counter("records.processed");
+    metricGrp.gauge("record.processing.time.ts", (Gauge<Long>) () -> _lastRecordProcessingTimeMs);
+  }
+
+  private void resetBuffer()
+      throws IOException {
+    FileUtils.deleteQuietly(_bufferFile);
+    _rowCount = 0;
+    _recordWriter = new DataFileWriter<>(new GenericDatumWriter<>(_avroSchema));
+    _recordWriter.create(_avroSchema, _bufferFile);
+  }
+
+  @Override
+  public void collect(GenericRow row)
+      throws IOException {
+    long startTime = System.currentTimeMillis();
+    GenericRow transform = _recordTransformer.transform(row);
+    SegmentProcessorAvroUtils.convertGenericRowToAvroRecord(transform, _reusableRecord, _fieldsToRead);
+    _rowCount++;
+    _recordWriter.append(_reusableRecord);
+    _lastRecordProcessingTimeMs = System.currentTimeMillis() - startTime;
+    _processedRecords.inc();
+  }
+
+  /**
+   * Creates one Pinot segment using the {@link GenericRow}s collected in the AVRO file buffer, at
+   * the outputDirUri as specified in the tableConfig->batchConfigs. Successful invocation of this
+   * method means that the {@link GenericRow}s collected so far, are now available in the Pinot
+   * segment and not available in the buffer anymore.
+   *
+   * <p>Successful completion of segment will return the segment URI, and the URI includes a
+   * sequence id indicating the part number. The sequence id is initialized to 0 and each successful
+   * flush will increment the sequence id by 1. The buffer will be reset and ready to accept further
+   * records via <code>collect()</code> If an exception is thrown, the buffer will not be reset and
+   * so, <code>flush()</code> can be invoked repeatedly in a retry loop. If a successful invocation
+   * is not achieved,<code>close()</code> followed by <code>init </code> will have to be called in
+   * order to reset the buffer and resume record writing.
+   *
+   * @return URI of the generated segment
+   * @throws IOException
+   */
+  @Override
+  public URI flush()
+      throws IOException {
+
+    LOGGER.info("Beginning flush for Pinot table: {} with {} records", _tableNameWithType, _rowCount);
+    _recordWriter.close();
+
+    // Create temp dir for flush
+    File flushDir = new File(_stagingDir, "flush_dir_" + System.currentTimeMillis());
+    Preconditions.checkState(flushDir.mkdirs(), "Failed to create flush dir: %s", flushDir);
+
+    try {
+      // Segment dir
+      File segmentDir = new File(flushDir, "segment_dir");
+
+      // Make BatchIngestionConfig for flush
+      Map<String, String> batchConfigMapOverride = new HashMap<>(_batchConfig.getBatchConfigMap());
+      batchConfigMapOverride.put(BatchConfigProperties.INPUT_DIR_URI, _bufferFile.getAbsolutePath());
+      batchConfigMapOverride.put(BatchConfigProperties.OUTPUT_DIR_URI, segmentDir.getAbsolutePath());
+      batchConfigMapOverride.put(BatchConfigProperties.INPUT_FORMAT, BUFFER_FILE_FORMAT.toString());
+      BatchIngestionConfig batchIngestionConfig = new BatchIngestionConfig(Lists.newArrayList(batchConfigMapOverride),
+          _batchIngestionConfig.getSegmentIngestionType(), _batchIngestionConfig.getSegmentIngestionFrequency());
+
+      // Build segment
+      SegmentGeneratorConfig segmentGeneratorConfig =
+          IngestionUtils.generateSegmentGeneratorConfig(_tableConfig, _schema, batchIngestionConfig);
+      segmentGeneratorConfig.setSequenceId(_seqId);
+      String segmentName = IngestionUtils.buildSegment(segmentGeneratorConfig);
+      LOGGER.info("Successfully built segment: {} of sequence {} for Pinot table: {}", segmentName, _seqId,
+          _tableNameWithType);
+
+      // Tar segment
+      File segmentTarFile =
+          new File(_outputDirURI, String.format("%s_%d%s", segmentName, _indexOfSubtask, Constants.TAR_GZ_FILE_EXT));

Review comment:
       ah, I see your point. Yes, you are right. Updated.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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