You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@bahir.apache.org by GitBox <gi...@apache.org> on 2021/03/10 19:25:53 UTC

[GitHub] [bahir-flink] mschroederi opened a new pull request #113: Apache Pinot Connector Sink

mschroederi opened a new pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113


   This MR implements an Apache Pinot connector sink using the new sink interface introduced in [FLIP-143](https://cwiki.apache.org/confluence/display/FLINK/FLIP-143%3A+Unified+Sink+API#FLIP143:UnifiedSinkAPI-SinkAPI).


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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r596179035



##########
File path: flink-connector-pinot/pom.xml
##########
@@ -0,0 +1,225 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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/maven-v4_0_0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.bahir</groupId>
+        <artifactId>bahir-flink-parent_2.11</artifactId>
+        <version>1.1-SNAPSHOT</version>
+        <relativePath>..</relativePath>
+    </parent>
+
+    <artifactId>flink-connector-pinot_2.11</artifactId>
+    <name>flink-connector-pinot</name>
+
+
+    <packaging>jar</packaging>
+
+    <!-- Allow users to pass custom connector versions -->
+    <properties>
+        <pinot.version>0.6.0</pinot.version>
+
+        <testcontainers.version>1.15.2</testcontainers.version>

Review comment:
       You're right. Thanks for noting 👍




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594710581



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();

Review comment:
       Nonetheless, I guess the threading model makes the `PinotSink` much more usable. I've started some benchmarks and saw a huge increase in throughput while keeping the event time latency stable in comparison to the single threaded `PinotSinkGlobalCommitter`.




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594310198



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {

Review comment:
       good hint. Thanks 👍




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594695051



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotWriterSegment.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@link PinotWriterSegment} represents exactly one segment that can be found in the Pinot
+ * cluster once the commit has been completed.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotWriterSegment<IN> implements Serializable {
+
+    private static final Logger LOG = LoggerFactory.getLogger("PinotWriterSegment");
+
+    private final int maxRowsPerSegment;
+    private final String tempDirPrefix;
+    private final JsonSerializer<IN> jsonSerializer;
+    private final FileSystemAdapter fsAdapter;
+
+    private boolean acceptsElements = true;
+
+    private final List<IN> elements;
+    private File dataFile;
+    private long minTimestamp = Long.MAX_VALUE;
+    private long maxTimestamp = Long.MIN_VALUE;
+
+    /**
+     * @param maxRowsPerSegment Maximum number of rows to be stored within a Pinot segment
+     * @param tempDirPrefix     Prefix for temp directories used
+     * @param jsonSerializer    Serializer used to convert elements to JSON
+     * @param fsAdapter         Filesystem adapter used to save files for sharing files across nodes
+     */
+    protected PinotWriterSegment(int maxRowsPerSegment, String tempDirPrefix, JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        checkArgument(maxRowsPerSegment > 0L);
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.elements = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements and stores them in memory until either {@link #maxRowsPerSegment} is reached
+     * or {@link #prepareCommit} is called.
+     *
+     * @param element   Object from upstream task
+     * @param timestamp Timestamp assigned to element
+     * @throws IOException
+     */
+    public void write(IN element, long timestamp) throws IOException {
+        if (!this.acceptsElements()) {
+            throw new IllegalStateException("This PinotSegmentWriter does not accept any elements anymore.");
+        }
+        this.elements.add(element);
+        this.minTimestamp = Long.min(this.minTimestamp, timestamp);
+        this.maxTimestamp = Long.max(this.maxTimestamp, timestamp);
+
+        // Writes elements to local filesystem once the maximum number of items is reached
+        if (this.elements.size() == this.maxRowsPerSegment) {

Review comment:
       This is now implemented in the current version of this PR. The `FileSystemAdapter` is now capable of writing and reading directly from the shared filesystem. Moreover, it supports deleting files once the committables were successfully committed.




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

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



[GitHub] [bahir-flink] fapaul commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
fapaul commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r596899265



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import com.google.common.collect.Iterables;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Accepts incoming elements and creates {@link PinotSinkCommittable}s out of them on request.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotSinkWriter<IN> implements SinkWriter<IN, PinotSinkCommittable, Void> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkWriter.class);
+
+    private final int maxRowsPerSegment;
+    private final EventTimeExtractor<IN> eventTimeExtractor;
+    private final JsonSerializer<IN> jsonSerializer;
+
+    private final List<PinotWriterSegment<IN>> activeSegments;
+    private final FileSystemAdapter fsAdapter;
+
+    private final int subtaskId;
+
+    /**
+     * @param subtaskId          Subtask id provided by Flink
+     * @param maxRowsPerSegment  Maximum number of rows to be stored within a Pinot segment
+     * @param eventTimeExtractor Defines the way event times are extracted from received objects
+     * @param jsonSerializer     Serializer used to convert elements to JSON
+     * @param fsAdapter          Filesystem adapter used to save files for sharing files across nodes
+     */
+    public PinotSinkWriter(int subtaskId, int maxRowsPerSegment,
+                           EventTimeExtractor<IN> eventTimeExtractor,
+                           JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        this.subtaskId = subtaskId;
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.eventTimeExtractor = checkNotNull(eventTimeExtractor);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.activeSegments = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements from an upstream tasks and writes them into {@link PinotWriterSegment}
+     *
+     * @param element Object from upstream task
+     * @param context SinkWriter context
+     * @throws IOException
+     */
+    @Override
+    public void write(IN element, Context context) throws IOException {
+        final PinotWriterSegment<IN> inProgressSegment = getOrCreateInProgressSegment();
+        inProgressSegment.write(element, eventTimeExtractor.getEventTime(element, context));
+    }
+
+    /**
+     * Creates {@link PinotSinkCommittable}s from elements previously received via {@link #write}.
+     * If flush is set, all {@link PinotWriterSegment}s are transformed into
+     * {@link PinotSinkCommittable}s. If flush is not set, only currently non-active
+     * {@link PinotSinkCommittable}s are transformed into {@link PinotSinkCommittable}s.
+     * To convert a {@link PinotWriterSegment} into a {@link PinotSinkCommittable} the data gets
+     * written to the shared filesystem. Moreover, minimum and maximum timestamps are identified.
+     * Finally, all {@link PinotWriterSegment}s transformed into {@link PinotSinkCommittable}s are
+     * removed from {@link #activeSegments}.
+     *
+     * @param flush Flush all currently known elements into the {@link PinotSinkCommittable}s
+     * @return List of {@link PinotSinkCommittable} to process in {@link org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter}
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkCommittable> prepareCommit(boolean flush) throws IOException {
+        // Identify segments to commit. If the flush argument is set all segments shall be committed.
+        // Otherwise, take only those PinotWriterSegments that do not accept any more elements.
+        List<PinotWriterSegment<IN>> segmentsToCommit = activeSegments.stream()
+                .filter(s -> flush || !s.acceptsElements())
+                .collect(Collectors.toList());
+        LOG.debug("Identified {} segments to commit [subtaskId={}]", segmentsToCommit.size(), subtaskId);
+
+        LOG.debug("Creating committables... [subtaskId={}]", subtaskId);
+        List<PinotSinkCommittable> committables = new ArrayList<>();
+        for (final PinotWriterSegment<IN> segment : segmentsToCommit) {
+            committables.add(segment.prepareCommit());
+        }
+        LOG.debug("Created {} committables [subtaskId={}]", committables.size(), subtaskId);
+
+        // Remove all PinotWriterSegments that will be emitted within the committables.
+        activeSegments.removeAll(segmentsToCommit);
+        return committables;
+    }
+
+    /**
+     * Gets the {@link PinotWriterSegment} still accepting elements or creates a new one.
+     *
+     * @return {@link PinotWriterSegment} accepting at least one more element
+     */
+    private PinotWriterSegment<IN> getOrCreateInProgressSegment() {
+        final PinotWriterSegment<IN> latestSegment = Iterables.getLast(activeSegments, null);
+        if (latestSegment == null || !latestSegment.acceptsElements()) {
+            final PinotWriterSegment<IN> inProgressSegment = new PinotWriterSegment<>(maxRowsPerSegment, jsonSerializer, fsAdapter);
+            activeSegments.add(inProgressSegment);
+            return inProgressSegment;
+        }
+        return latestSegment;
+    }
+
+    /**
+     * As we do not need to save any information in snapshots,
+     * this method always returns an empty ArrayList.
+     *
+     * @return always an empty ArrayList
+     */
+    @Override
+    public List<Void> snapshotState() {

Review comment:
       I was thinking about the following scenario:
   
   The `PinotSinkWriter` prepares for a checkpoint and calls `prepareCommit` now some of the records are not published because the segment is not full. So `activeSegments` contains records which we have received in this checkpoint period of before. With a checkpoint, all operators acknowledge that they can recover the state they had at the last checkpoint. In the current situation, this seems not to be possible because the records in `activeSegments` are essentially lost.
   
   You are right it is not ensured that the same subtask receives the same messages **but** the state and the corresponding messages are always colocated. 
   
   I think you have to write all unfinished active segments into the state to potentially recover from. Otherwise, this is only at-most-once delivery. WDYT? We can also discuss this again offline if you want.




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r592796013



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,428 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+                e.printStackTrace();
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        public CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    static class SegmentCommitter implements Callable<Boolean> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class);
+
+        final String pinotControllerHost;
+        final String pinotControllerPort;
+        final String tempDirPrefix;
+        final FileSystemAdapter fsAdapter;
+        final String dataFilePath;
+        final String segmentName;
+        final Schema tableSchema;
+        final TableConfig tableConfig;
+        final String timeColumnName;
+        final TimeUnit segmentTimeUnit;
+
+        /**
+         * @param pinotControllerHost Host of the Pinot controller
+         * @param pinotControllerPort Port of the Pinot controller
+         * @param fsAdapter           Filesystem adapter used to load data files from the shared file system
+         * @param dataFilePath        Data file to load from the shared file system
+         * @param segmentName         Name of the segment to create and commit
+         * @param tableSchema         Pinot table schema
+         * @param tableConfig         Pinot table config
+         * @param timeColumnName      Name of the column containing the timestamp
+         * @param segmentTimeUnit     Unit of the time column
+         */
+        public SegmentCommitter(String pinotControllerHost, String pinotControllerPort, String tempDirPrefix, FileSystemAdapter fsAdapter, String dataFilePath, String segmentName, Schema tableSchema, TableConfig tableConfig, String timeColumnName, TimeUnit segmentTimeUnit) {
+            this.pinotControllerHost = pinotControllerHost;
+            this.pinotControllerPort = pinotControllerPort;
+            this.tempDirPrefix = tempDirPrefix;
+            this.fsAdapter = fsAdapter;
+            this.dataFilePath = dataFilePath;
+            this.segmentName = segmentName;
+            this.tableSchema = tableSchema;
+            this.tableConfig = tableConfig;
+            this.timeColumnName = timeColumnName;
+            this.segmentTimeUnit = segmentTimeUnit;
+        }
+
+        /**
+         * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment
+         * and finally uploads the segment to the Pinot controller
+         *
+         * @return True if the commit succeeded
+         */
+        @Override
+        public Boolean call() {
+            try {
+                // Download data file from the shared filesystem
+                LOG.info("Downloading data file {} from shared file system...", dataFilePath);
+                File segmentData = fsAdapter.copyToLocalFile(dataFilePath);
+                LOG.info("Successfully downloaded data file {} from shared file system", dataFilePath);
+
+                File segmentFile = Files.createTempDirectory(this.tempDirPrefix).toFile();
+                LOG.info("Creating segment in " + segmentFile.getAbsolutePath());
+
+                // Creates a segment with name `segmentName` in `segmentFile`
+                this.generateSegment(segmentData, segmentFile, true);
+
+                // Uploads the recently created segment to the Pinot controller
+                this.uploadSegment(segmentFile);
+
+                // Commit successful
+                return true;
+            } catch (IOException e) {
+                e.printStackTrace();

Review comment:
       done




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r592795601



##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/emulator/PinotEmulatorManager.java
##########
@@ -0,0 +1,341 @@
+/*
+ * 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.flink.streaming.connectors.pinot.emulator;
+
+import com.github.dockerjava.api.DockerClient;

Review comment:
       Great advice, the manual testing setup was replaced by testcontainers




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r592795025



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.HttpEntity;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHost;
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        this.controllerHost = checkNotNull(controllerHost);

Review comment:
       You're right, thanks for noting




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

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



[GitHub] [bahir-flink] eskabetxe commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
eskabetxe commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r604244729



##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.util.TestLogger;
+import org.apache.pinot.spi.config.table.*;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.HttpWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.time.Duration;
+
+/**
+ * Base class for PinotSink e2e tests
+ */
+@Testcontainers
+public class PinotTestBase extends TestLogger {
+
+    protected static final Logger LOG = LoggerFactory.getLogger(PinotTestBase.class);
+
+    private static final String DOCKER_IMAGE_NAME = "apachepinot/pinot:0.6.0";
+    private static final Integer PINOT_INTERNAL_BROKER_PORT = 8000;
+    private static final Integer PINOT_INTERNAL_CONTROLLER_PORT = 9000;
+
+    protected static TableConfig TABLE_CONFIG;
+    protected static final Schema TABLE_SCHEMA = PinotTableConfig.getTableSchema();
+    protected static PinotTestHelper pinotHelper;
+
+    /**
+     * Creates the Pinot testcontainer. We delay the start of tests until Pinot has started all
+     * internal components. This is identified through a log statement.
+     */
+    @Container
+    public static GenericContainer<?> pinot = new GenericContainer<>(DockerImageName.parse(DOCKER_IMAGE_NAME))
+            .withCommand("QuickStart", "-type", "batch")
+            .withExposedPorts(PINOT_INTERNAL_BROKER_PORT, PINOT_INTERNAL_CONTROLLER_PORT)
+            .waitingFor(
+                    // Wait for controller, server and broker instances to be available
+                    new HttpWaitStrategy()
+                            .forPort(PINOT_INTERNAL_CONTROLLER_PORT)
+                            .forPath("/instances")
+                            .forStatusCode(200)
+                            .forResponsePredicate(res -> {
+                                try {
+                                    JsonNode instances = JsonUtils.stringToJsonNode(res).get("instances");
+                                    // Expect 3 instances to be up and running (controller, broker and server)
+                                    return instances.size() == 3;
+                                } catch (IOException e) {
+                                    LOG.error("Error while reading json response in HttpWaitStrategy.", e);
+                                }
+                                return false;
+                            })
+                            // Allow Pinot to take up to 180s for starting up
+                            .withStartupTimeout(Duration.ofSeconds(180))
+            );
+
+    /**
+     * Creates a new instance of the {@link PinotTestHelper} using the testcontainer port mappings
+     * and creates the test table.
+     *
+     * @throws IOException
+     */
+    @BeforeEach
+    public void setUp() throws IOException {
+        TABLE_CONFIG = PinotTableConfig.getTableConfig();
+        pinotHelper = new PinotTestHelper(getPinotHost(), getPinotControllerPort(), getPinotBrokerPort());
+        pinotHelper.createTable(TABLE_CONFIG, TABLE_SCHEMA);
+    }
+
+    /**
+     * Delete the test table after each test.
+     *
+     * @throws Exception
+     */
+    @AfterEach
+    public void tearDown() throws Exception {
+        pinotHelper.deleteTable(TABLE_CONFIG, TABLE_SCHEMA);
+    }
+
+    /**
+     * Returns the current Pinot table name
+     *
+     * @return Pinot table name
+     */
+    protected String getTableName() {
+        return TABLE_CONFIG.getTableName();
+    }
+
+    /**
+     * Returns the host the Pinot container is available at
+     *
+     * @return Pinot container host
+     */
+    protected String getPinotHost() {
+        return pinot.getHost();
+    }
+
+
+    /**
+     * Returns the Pinot controller port from the container ports.
+     *
+     * @return Pinot controller port
+     */
+    protected String getPinotControllerPort() {
+        return pinot.getMappedPort(PINOT_INTERNAL_CONTROLLER_PORT).toString();
+    }
+
+    /**
+     * Returns the Pinot broker port from the container ports.
+     *
+     * @return Pinot broker port
+     */
+    private String getPinotBrokerPort() {
+        return pinot.getMappedPort(PINOT_INTERNAL_BROKER_PORT).toString();
+    }
+
+    /**
+     * Class defining the elements passed to the {@link PinotSink} during the tests.
+     */
+    protected static class SingleColumnTableRow {
+
+        private String _col1;
+        private Long _timestamp;
+
+        SingleColumnTableRow(@JsonProperty(value = "col1", required = true) String col1,
+                             @JsonProperty(value = "timestamp", required = true) Long timestamp) {
+            this._col1 = col1;
+            this._timestamp = timestamp;
+        }
+
+        @JsonProperty("col1")
+        public String getCol1() {
+            return this._col1;
+        }
+
+        public void setCol1(String _col1) {
+            this._col1 = _col1;
+        }
+
+        @JsonProperty("timestamp")
+        public Long getTimestamp() {
+            return this._timestamp;
+        }
+
+        public void setTimestamp(Long timestamp) {
+            this._timestamp = timestamp;
+        }
+    }
+
+    /**
+     * Serializes {@link SingleColumnTableRow} to JSON.
+     */
+    protected static class SingleColumnTableRowSerializer extends JsonSerializer<SingleColumnTableRow> {
+
+        @Override
+        public String toJson(SingleColumnTableRow element) {
+            return JsonUtils.objectToJsonNode(element).toString();
+        }
+    }
+
+    /**
+     * Pinot table configuration helpers.
+     */
+    private static class PinotTableConfig {
+
+        static final String TABLE_NAME_PREFIX = "FLTable";
+        static final String SCHEMA_NAME = "FLTableSchema";
+
+        private static SegmentsValidationAndRetentionConfig getValidationConfig() {
+            SegmentsValidationAndRetentionConfig validationConfig = new SegmentsValidationAndRetentionConfig();
+            validationConfig.setSegmentAssignmentStrategy("BalanceNumSegmentAssignmentStrategy");
+            validationConfig.setSegmentPushType("APPEND");
+            validationConfig.setSchemaName(SCHEMA_NAME);
+            validationConfig.setReplication("1");
+            return validationConfig;
+        }
+
+        private static TenantConfig getTenantConfig() {
+            TenantConfig tenantConfig = new TenantConfig("DefaultTenant", "DefaultTenant", null);
+            return tenantConfig;
+        }
+
+        private static IndexingConfig getIndexingConfig() {
+            IndexingConfig indexingConfig = new IndexingConfig();
+            return indexingConfig;
+        }
+
+        private static TableCustomConfig getCustomConfig() {
+            TableCustomConfig customConfig = new TableCustomConfig(null);
+            ;

Review comment:
       abandoned ';'




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r597984304



##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotSinkTest.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.pinot.client.ResultSet;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.opentest4j.AssertionFailedError;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * E2e tests for Pinot Sink using BATCH and STREAMING execution mode
+ */
+public class PinotSinkTest extends PinotTestBase {
+
+    /**
+     * Tests the BATCH execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testBatchSink() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.BATCH);
+        env.setParallelism(2);
+
+        List<SingleColumnTableRow> data = getTestData(12);
+        this.setupDataStream(env, data);
+
+        // Run
+        executeOnMiniCluster(env.getStreamGraph().getJobGraph());
+
+        checkForDataInPinotWithRetry(data, 20);
+    }
+
+    /**
+     * Tests the STREAMING execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testStreamingSink() throws Exception {
+        final Configuration conf = new Configuration();
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(conf);
+        env.setRuntimeMode(RuntimeExecutionMode.STREAMING);
+        env.setParallelism(2);
+        env.enableCheckpointing(50);
+
+        List<SingleColumnTableRow> data = getTestData(12);
+        this.setupDataStream(env, data);
+
+        // Run
+        executeOnMiniCluster(env.getStreamGraph().getJobGraph());
+
+        // We only expect the first 100 elements to be already committed to Pinot.
+        // The remaining would follow once we increase the input data size.
+        // The stream executions stops once the last input tuple was sent to the sink.
+        checkForDataInPinotWithRetry(data, 20);
+    }
+
+    /**
+     * Generates a small test dataset consisting of {@link SingleColumnTableRow}s.
+     *
+     * @return List of SingleColumnTableRow
+     */
+    private List<SingleColumnTableRow> getTestData(int numItems) {
+        return IntStream.range(1, numItems + 1)
+                .mapToObj(num -> "ColValue" + num)
+                .map(col1 -> new SingleColumnTableRow(col1, System.currentTimeMillis()))
+                .collect(Collectors.toList());
+    }
+
+    /**
+     * Executes a given JobGraph on a MiniCluster.
+     *
+     * @param jobGraph JobGraph to execute
+     * @throws Exception
+     */
+    private void executeOnMiniCluster(JobGraph jobGraph) throws Exception {
+        final Configuration config = new Configuration();
+        config.setString(RestOptions.BIND_PORT, "18081-19000");
+        final MiniClusterConfiguration cfg =

Review comment:
       Great hint!




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r596189490



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import com.google.common.collect.Iterables;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Accepts incoming elements and creates {@link PinotSinkCommittable}s out of them on request.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotSinkWriter<IN> implements SinkWriter<IN, PinotSinkCommittable, Void> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkWriter.class);
+
+    private final int maxRowsPerSegment;
+    private final EventTimeExtractor<IN> eventTimeExtractor;
+    private final JsonSerializer<IN> jsonSerializer;
+
+    private final List<PinotWriterSegment<IN>> activeSegments;
+    private final FileSystemAdapter fsAdapter;
+
+    private final int subtaskId;
+
+    /**
+     * @param subtaskId          Subtask id provided by Flink
+     * @param maxRowsPerSegment  Maximum number of rows to be stored within a Pinot segment
+     * @param eventTimeExtractor Defines the way event times are extracted from received objects
+     * @param jsonSerializer     Serializer used to convert elements to JSON
+     * @param fsAdapter          Filesystem adapter used to save files for sharing files across nodes
+     */
+    public PinotSinkWriter(int subtaskId, int maxRowsPerSegment,
+                           EventTimeExtractor<IN> eventTimeExtractor,
+                           JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        this.subtaskId = subtaskId;
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.eventTimeExtractor = checkNotNull(eventTimeExtractor);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.activeSegments = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements from an upstream tasks and writes them into {@link PinotWriterSegment}
+     *
+     * @param element Object from upstream task
+     * @param context SinkWriter context
+     * @throws IOException
+     */
+    @Override
+    public void write(IN element, Context context) throws IOException {
+        final PinotWriterSegment<IN> inProgressSegment = getOrCreateInProgressSegment();
+        inProgressSegment.write(element, eventTimeExtractor.getEventTime(element, context));
+    }
+
+    /**
+     * Creates {@link PinotSinkCommittable}s from elements previously received via {@link #write}.
+     * If flush is set, all {@link PinotWriterSegment}s are transformed into
+     * {@link PinotSinkCommittable}s. If flush is not set, only currently non-active
+     * {@link PinotSinkCommittable}s are transformed into {@link PinotSinkCommittable}s.
+     * To convert a {@link PinotWriterSegment} into a {@link PinotSinkCommittable} the data gets
+     * written to the shared filesystem. Moreover, minimum and maximum timestamps are identified.
+     * Finally, all {@link PinotWriterSegment}s transformed into {@link PinotSinkCommittable}s are
+     * removed from {@link #activeSegments}.
+     *
+     * @param flush Flush all currently known elements into the {@link PinotSinkCommittable}s
+     * @return List of {@link PinotSinkCommittable} to process in {@link org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter}
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkCommittable> prepareCommit(boolean flush) throws IOException {
+        // Identify segments to commit. If the flush argument is set all segments shall be committed.
+        // Otherwise, take only those PinotWriterSegments that do not accept any more elements.
+        List<PinotWriterSegment<IN>> segmentsToCommit = activeSegments.stream()
+                .filter(s -> flush || !s.acceptsElements())
+                .collect(Collectors.toList());
+        LOG.debug("Identified {} segments to commit [subtaskId={}]", segmentsToCommit.size(), subtaskId);
+
+        LOG.debug("Creating committables... [subtaskId={}]", subtaskId);
+        List<PinotSinkCommittable> committables = new ArrayList<>();
+        for (final PinotWriterSegment<IN> segment : segmentsToCommit) {
+            committables.add(segment.prepareCommit());
+        }
+        LOG.debug("Created {} committables [subtaskId={}]", committables.size(), subtaskId);
+
+        // Remove all PinotWriterSegments that will be emitted within the committables.
+        activeSegments.removeAll(segmentsToCommit);
+        return committables;
+    }
+
+    /**
+     * Gets the {@link PinotWriterSegment} still accepting elements or creates a new one.
+     *
+     * @return {@link PinotWriterSegment} accepting at least one more element
+     */
+    private PinotWriterSegment<IN> getOrCreateInProgressSegment() {
+        final PinotWriterSegment<IN> latestSegment = Iterables.getLast(activeSegments, null);
+        if (latestSegment == null || !latestSegment.acceptsElements()) {
+            final PinotWriterSegment<IN> inProgressSegment = new PinotWriterSegment<>(maxRowsPerSegment, jsonSerializer, fsAdapter);
+            activeSegments.add(inProgressSegment);
+            return inProgressSegment;
+        }
+        return latestSegment;
+    }
+
+    /**
+     * As we do not need to save any information in snapshots,
+     * this method always returns an empty ArrayList.
+     *
+     * @return always an empty ArrayList
+     */
+    @Override
+    public List<Void> snapshotState() {

Review comment:
       Not sure why we would need to store anything here. Could you elaborate on what you consider to be 'active files' ?
   From my understanding we cannot reuse previously written `PinotWriterSegment`s, as elements might get sent to different subTasks when recovering from failure.




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r597274483



##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotSinkTest.java
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.pinot.client.ResultSet;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.opentest4j.AssertionFailedError;
+
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * E2e tests for Pinot Sink using BATCH and STREAMING execution mode
+ */
+public class PinotSinkTest extends PinotTestBase {
+
+    /**
+     * Tests the BATCH execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testBatchSink() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.BATCH);
+        env.setParallelism(2);
+
+        List<SingleColumnTableRow> data = getTestData(12);
+        this.setupDataStream(env, data);
+
+        // Run
+        env.execute();
+
+        checkForDataInPinotWithRetry(data, data.size(), 20);
+    }
+
+    /**
+     * Tests the STREAMING execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testStreamingSink() throws Exception {
+        final Configuration conf = new Configuration();
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(conf);
+        env.setRuntimeMode(RuntimeExecutionMode.STREAMING);
+        env.setParallelism(2);
+        env.enableCheckpointing(50);
+
+        List<SingleColumnTableRow> data = getTestData(1000);
+        this.setupDataStream(env, data);
+
+        // Run
+        env.execute();
+
+        // We only expect the first 100 elements to be already committed to Pinot.
+        // The remaining would follow once we increase the input data size.
+        // The stream executions stops once the last input tuple was sent to the sink.
+        checkForDataInPinotWithRetry(data, 100, 20);

Review comment:
       I just saw that I missed the comment on the `MiniCluster` execution from the last review cycle. After finally switching to that execution form, everything works fine - including that the jobs are only stopped once all elements were committed.




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594302050



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        public CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    static class SegmentCommitter implements Callable<Boolean> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class);
+
+        final String pinotControllerHost;
+        final String pinotControllerPort;
+        final String tempDirPrefix;
+        final FileSystemAdapter fsAdapter;
+        final String dataFilePath;
+        final String segmentName;
+        final Schema tableSchema;
+        final TableConfig tableConfig;
+        final String timeColumnName;
+        final TimeUnit segmentTimeUnit;
+
+        /**
+         * @param pinotControllerHost Host of the Pinot controller
+         * @param pinotControllerPort Port of the Pinot controller
+         * @param fsAdapter           Filesystem adapter used to load data files from the shared file system
+         * @param dataFilePath        Data file to load from the shared file system
+         * @param segmentName         Name of the segment to create and commit
+         * @param tableSchema         Pinot table schema
+         * @param tableConfig         Pinot table config
+         * @param timeColumnName      Name of the column containing the timestamp
+         * @param segmentTimeUnit     Unit of the time column
+         */
+        public SegmentCommitter(String pinotControllerHost, String pinotControllerPort, String tempDirPrefix, FileSystemAdapter fsAdapter, String dataFilePath, String segmentName, Schema tableSchema, TableConfig tableConfig, String timeColumnName, TimeUnit segmentTimeUnit) {
+            this.pinotControllerHost = pinotControllerHost;
+            this.pinotControllerPort = pinotControllerPort;
+            this.tempDirPrefix = tempDirPrefix;
+            this.fsAdapter = fsAdapter;
+            this.dataFilePath = dataFilePath;
+            this.segmentName = segmentName;
+            this.tableSchema = tableSchema;
+            this.tableConfig = tableConfig;
+            this.timeColumnName = timeColumnName;
+            this.segmentTimeUnit = segmentTimeUnit;
+        }
+
+        /**
+         * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment
+         * and finally uploads the segment to the Pinot controller
+         *
+         * @return True if the commit succeeded
+         */
+        @Override
+        public Boolean call() {
+            try {
+                // Download data file from the shared filesystem
+                LOG.info("Downloading data file {} from shared file system...", dataFilePath);

Review comment:
       now using debug instead of info all over the place

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        public CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    static class SegmentCommitter implements Callable<Boolean> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class);
+
+        final String pinotControllerHost;
+        final String pinotControllerPort;
+        final String tempDirPrefix;
+        final FileSystemAdapter fsAdapter;
+        final String dataFilePath;
+        final String segmentName;
+        final Schema tableSchema;
+        final TableConfig tableConfig;
+        final String timeColumnName;
+        final TimeUnit segmentTimeUnit;
+
+        /**
+         * @param pinotControllerHost Host of the Pinot controller
+         * @param pinotControllerPort Port of the Pinot controller
+         * @param fsAdapter           Filesystem adapter used to load data files from the shared file system
+         * @param dataFilePath        Data file to load from the shared file system
+         * @param segmentName         Name of the segment to create and commit
+         * @param tableSchema         Pinot table schema
+         * @param tableConfig         Pinot table config
+         * @param timeColumnName      Name of the column containing the timestamp
+         * @param segmentTimeUnit     Unit of the time column
+         */
+        public SegmentCommitter(String pinotControllerHost, String pinotControllerPort, String tempDirPrefix, FileSystemAdapter fsAdapter, String dataFilePath, String segmentName, Schema tableSchema, TableConfig tableConfig, String timeColumnName, TimeUnit segmentTimeUnit) {
+            this.pinotControllerHost = pinotControllerHost;
+            this.pinotControllerPort = pinotControllerPort;
+            this.tempDirPrefix = tempDirPrefix;
+            this.fsAdapter = fsAdapter;
+            this.dataFilePath = dataFilePath;
+            this.segmentName = segmentName;
+            this.tableSchema = tableSchema;
+            this.tableConfig = tableConfig;
+            this.timeColumnName = timeColumnName;
+            this.segmentTimeUnit = segmentTimeUnit;
+        }
+
+        /**
+         * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment
+         * and finally uploads the segment to the Pinot controller
+         *
+         * @return True if the commit succeeded
+         */
+        @Override
+        public Boolean call() {
+            try {
+                // Download data file from the shared filesystem
+                LOG.info("Downloading data file {} from shared file system...", dataFilePath);
+                File segmentData = fsAdapter.copyToLocalFile(dataFilePath);
+                LOG.info("Successfully downloaded data file {} from shared file system", dataFilePath);
+
+                File segmentFile = Files.createTempDirectory(this.tempDirPrefix).toFile();
+                LOG.info("Creating segment in " + segmentFile.getAbsolutePath());
+
+                // Creates a segment with name `segmentName` in `segmentFile`
+                this.generateSegment(segmentData, segmentFile, true);
+
+                // Uploads the recently created segment to the Pinot controller
+                this.uploadSegment(segmentFile);
+
+                // Commit successful
+                return true;
+            } catch (IOException e) {
+                LOG.error(e.getMessage());
+
+                // Commit failed
+                return false;
+            }
+        }
+
+        /**
+         * Creates a segment from the given parameters.
+         * This method was adapted from {@link org.apache.pinot.tools.admin.command.CreateSegmentCommand}.
+         *
+         * @param dataFile                  File containing the JSON data
+         * @param outDir                    Segment target path
+         * @param _postCreationVerification Verify segment after generation
+         */
+        public void generateSegment(File dataFile, File outDir, Boolean _postCreationVerification) {
+            SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(tableConfig, tableSchema);
+            segmentGeneratorConfig.setSegmentName(segmentName);
+            segmentGeneratorConfig.setSegmentTimeUnit(segmentTimeUnit);
+            segmentGeneratorConfig.setTimeColumnName(timeColumnName);
+            segmentGeneratorConfig.setInputFilePath(dataFile.getPath());
+            segmentGeneratorConfig.setFormat(FileFormat.JSON);
+            segmentGeneratorConfig.setOutDir(outDir.getPath());
+            segmentGeneratorConfig.setTableName(tableConfig.getTableName());
+
+            try {
+                SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+                driver.init(segmentGeneratorConfig);
+                driver.build();
+                File indexDir = new File(outDir, segmentName);
+                LOG.info("Successfully created segment: {} in directory: {}", segmentName, indexDir);

Review comment:
       now using debug instead of info all over the place




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r596178620



##########
File path: flink-connector-pinot/pom.xml
##########
@@ -0,0 +1,225 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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/maven-v4_0_0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.bahir</groupId>
+        <artifactId>bahir-flink-parent_2.11</artifactId>
+        <version>1.1-SNAPSHOT</version>
+        <relativePath>..</relativePath>
+    </parent>
+
+    <artifactId>flink-connector-pinot_2.11</artifactId>
+    <name>flink-connector-pinot</name>
+
+
+    <packaging>jar</packaging>
+
+    <!-- Allow users to pass custom connector versions -->
+    <properties>
+        <pinot.version>0.6.0</pinot.version>
+
+        <testcontainers.version>1.15.2</testcontainers.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-scala_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+        </dependency>
+
+        <!-- This is used to interact with Pinot Controller -->
+        <dependency>
+            <groupId>org.apache.pinot</groupId>
+            <artifactId>pinot-tools</artifactId>
+            <version>${pinot.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpmime</artifactId>
+            <version>4.5.13</version>
+        </dependency>
+
+        <!-- This is used to run the local Pinot -->
+        <dependency>
+            <groupId>com.github.docker-java</groupId>
+            <artifactId>docker-java</artifactId>
+            <version>3.2.7</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.github.docker-java</groupId>
+            <artifactId>docker-java-transport-httpclient5</artifactId>
+            <version>3.2.7</version>
+            <scope>test</scope>
+        </dependency>
+        <!-- This is used to interact with Pinot Broker -->
+        <dependency>
+            <groupId>org.apache.pinot</groupId>
+            <artifactId>pinot-java-client</artifactId>
+            <version>${pinot.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-test-utils_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-tests</artifactId>
+            <version>${flink.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-runtime_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-module-junit4</artifactId>
+            <version>1.5.5</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-api-mockito</artifactId>
+            <version>1.5.5</version>
+            <scope>test</scope>

Review comment:
       No they aren't. I'll remove them




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

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



[GitHub] [bahir-flink] fapaul commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
fapaul commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r598080032



##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java
##########
@@ -85,6 +92,35 @@
                             .withStartupTimeout(Duration.ofSeconds(180))
             );
 
+    @ClassRule

Review comment:
       You can either remove the `@ClassRule` or, my preference, use https://github.com/apache/flink/blob/master/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterWithClientResource.java. Currently the `@ClassRule` has no effect because the `MiniCluster` does not implement the required interfaces therefore you have to call `@BeforeAll` and `@AfterAll` manually. 

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java
##########
@@ -85,6 +92,35 @@
                             .withStartupTimeout(Duration.ofSeconds(180))
             );
 
+    @ClassRule
+    public static final MiniCluster MINI_CLUSTER = new MiniCluster(

Review comment:
       Either remove the `@ClassRule` or, my preference, use https://github.com/apache/flink/blob/master/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterWithClientResource.java. Currently the `@ClassRule` has no effect because the `MiniCluster` does not implement the required interfaces therefore you have to call `@BeforeAll` and `@AfterAll` manually.




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r595086726



##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.util.TestLogger;
+import org.apache.pinot.spi.config.table.*;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.Wait;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Base class for PinotSink e2e tests
+ */
+@Testcontainers
+public class PinotTestBase extends TestLogger implements Serializable {
+
+    public static final String DOCKER_IMAGE_NAME = "apachepinot/pinot:0.6.0";
+    public static final Integer PINOT_INTERNAL_BROKER_PORT = 8000;
+    public static final Integer PINOT_INTERNAL_CONTROLLER_PORT = 9000;
+
+    protected static final TableConfig TABLE_CONFIG = PinotTableConfig.getTableConfig();
+    protected static final String TABLE_NAME = TABLE_CONFIG.getTableName();
+    protected static final Schema TABLE_SCHEMA = PinotTableConfig.getTableSchema();
+    protected static PinotTestHelper pinotHelper;
+
+    /**
+     * Creates the Pinot testcontainer. We delay the start of tests until Pinot has started all
+     * internal components. This is identified through a log statement.
+     */
+    @Container
+    public GenericContainer<?> pinot = new GenericContainer<>(DockerImageName.parse(DOCKER_IMAGE_NAME))
+            .withCommand("QuickStart", "-type", "batch")
+            .withExposedPorts(PINOT_INTERNAL_BROKER_PORT, PINOT_INTERNAL_CONTROLLER_PORT)
+            .waitingFor(Wait
+                    .forLogMessage(".*You can always go to http://localhost:9000 to play around in the query console.*\\n", 1)

Review comment:
       We now make use of `HttpWaitStrategy` which checks via the public Pinot controller API whether all required instances in Pinot (Controller, Broker and Server) have been successfully startet up.




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

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



[GitHub] [bahir-flink] mschroederi commented on pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#issuecomment-812722160


   @eskabetxe Thanks for your feedback! Looking forward to seeing this 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.

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r592794973



##########
File path: flink-connector-pinot/pom.xml
##########
@@ -0,0 +1,213 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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/maven-v4_0_0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.bahir</groupId>
+        <artifactId>bahir-flink-parent_2.11</artifactId>
+        <version>1.1-SNAPSHOT</version>
+        <relativePath>..</relativePath>
+    </parent>
+
+    <artifactId>flink-connector-pinot_2.11</artifactId>
+    <name>flink-connector-pinot</name>
+
+
+    <packaging>jar</packaging>
+
+    <!-- Allow users to pass custom connector versions -->
+    <properties>
+        <pinot.version>0.6.0</pinot.version>
+
+        <!-- Flink version -->
+        <flink.version>1.12.0</flink.version>

Review comment:
       Totally right, this gets obsolete once #115 was 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.

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594718116



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);

Review comment:
       Totally right, `filterRecoveredCommittables` already handles the recovery and identified global committables that need to be re-committed, including the deleting of existing segment artifacts.




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594302483



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/segment/name/SimpleSegmentNameGenerator.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.flink.streaming.connectors.pinot.segment.name;
+
+import javax.annotation.Nullable;
+
+/**
+ * Adapted from {@link org.apache.pinot.core.segment.name.SimpleSegmentNameGenerator}.
+ * <p>
+ * Simple segment name generator which does not perform time conversion.
+ * <p>
+ * The segment name is simply joining the following fields with '_' but ignoring all the {@code null}s.
+ * <ul>
+ *   <li>Table name</li>
+ *   <li>Minimum time value</li>
+ *   <li>Maximum time value</li>
+ *   <li>Segment name postfix</li>
+ *   <li>Sequence id</li>
+ * </ul>
+ */
+public class SimpleSegmentNameGenerator extends PinotSinkSegmentNameGenerator {
+
+    private final String tableName;
+    private final String segmentNamePostfix;
+
+    public SimpleSegmentNameGenerator(String tableName, String segmentNamePostfix) {
+        this.tableName = tableName;
+        this.segmentNamePostfix = segmentNamePostfix;

Review comment:
       done




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594698360



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {

Review comment:
       Great hint! We now have a `PinotControllerClient` and `PinotControllerHttpClient`. The later provides an external API, e.g. used by the `PinotControllerClient`, to send http requests to the Pinot controller. The `PinotControllerClient` provides high-level functionality.




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

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



[GitHub] [bahir-flink] eskabetxe merged pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
eskabetxe merged pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113


   


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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r596189490



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import com.google.common.collect.Iterables;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Accepts incoming elements and creates {@link PinotSinkCommittable}s out of them on request.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotSinkWriter<IN> implements SinkWriter<IN, PinotSinkCommittable, Void> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkWriter.class);
+
+    private final int maxRowsPerSegment;
+    private final EventTimeExtractor<IN> eventTimeExtractor;
+    private final JsonSerializer<IN> jsonSerializer;
+
+    private final List<PinotWriterSegment<IN>> activeSegments;
+    private final FileSystemAdapter fsAdapter;
+
+    private final int subtaskId;
+
+    /**
+     * @param subtaskId          Subtask id provided by Flink
+     * @param maxRowsPerSegment  Maximum number of rows to be stored within a Pinot segment
+     * @param eventTimeExtractor Defines the way event times are extracted from received objects
+     * @param jsonSerializer     Serializer used to convert elements to JSON
+     * @param fsAdapter          Filesystem adapter used to save files for sharing files across nodes
+     */
+    public PinotSinkWriter(int subtaskId, int maxRowsPerSegment,
+                           EventTimeExtractor<IN> eventTimeExtractor,
+                           JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        this.subtaskId = subtaskId;
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.eventTimeExtractor = checkNotNull(eventTimeExtractor);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.activeSegments = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements from an upstream tasks and writes them into {@link PinotWriterSegment}
+     *
+     * @param element Object from upstream task
+     * @param context SinkWriter context
+     * @throws IOException
+     */
+    @Override
+    public void write(IN element, Context context) throws IOException {
+        final PinotWriterSegment<IN> inProgressSegment = getOrCreateInProgressSegment();
+        inProgressSegment.write(element, eventTimeExtractor.getEventTime(element, context));
+    }
+
+    /**
+     * Creates {@link PinotSinkCommittable}s from elements previously received via {@link #write}.
+     * If flush is set, all {@link PinotWriterSegment}s are transformed into
+     * {@link PinotSinkCommittable}s. If flush is not set, only currently non-active
+     * {@link PinotSinkCommittable}s are transformed into {@link PinotSinkCommittable}s.
+     * To convert a {@link PinotWriterSegment} into a {@link PinotSinkCommittable} the data gets
+     * written to the shared filesystem. Moreover, minimum and maximum timestamps are identified.
+     * Finally, all {@link PinotWriterSegment}s transformed into {@link PinotSinkCommittable}s are
+     * removed from {@link #activeSegments}.
+     *
+     * @param flush Flush all currently known elements into the {@link PinotSinkCommittable}s
+     * @return List of {@link PinotSinkCommittable} to process in {@link org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter}
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkCommittable> prepareCommit(boolean flush) throws IOException {
+        // Identify segments to commit. If the flush argument is set all segments shall be committed.
+        // Otherwise, take only those PinotWriterSegments that do not accept any more elements.
+        List<PinotWriterSegment<IN>> segmentsToCommit = activeSegments.stream()
+                .filter(s -> flush || !s.acceptsElements())
+                .collect(Collectors.toList());
+        LOG.debug("Identified {} segments to commit [subtaskId={}]", segmentsToCommit.size(), subtaskId);
+
+        LOG.debug("Creating committables... [subtaskId={}]", subtaskId);
+        List<PinotSinkCommittable> committables = new ArrayList<>();
+        for (final PinotWriterSegment<IN> segment : segmentsToCommit) {
+            committables.add(segment.prepareCommit());
+        }
+        LOG.debug("Created {} committables [subtaskId={}]", committables.size(), subtaskId);
+
+        // Remove all PinotWriterSegments that will be emitted within the committables.
+        activeSegments.removeAll(segmentsToCommit);
+        return committables;
+    }
+
+    /**
+     * Gets the {@link PinotWriterSegment} still accepting elements or creates a new one.
+     *
+     * @return {@link PinotWriterSegment} accepting at least one more element
+     */
+    private PinotWriterSegment<IN> getOrCreateInProgressSegment() {
+        final PinotWriterSegment<IN> latestSegment = Iterables.getLast(activeSegments, null);
+        if (latestSegment == null || !latestSegment.acceptsElements()) {
+            final PinotWriterSegment<IN> inProgressSegment = new PinotWriterSegment<>(maxRowsPerSegment, jsonSerializer, fsAdapter);
+            activeSegments.add(inProgressSegment);
+            return inProgressSegment;
+        }
+        return latestSegment;
+    }
+
+    /**
+     * As we do not need to save any information in snapshots,
+     * this method always returns an empty ArrayList.
+     *
+     * @return always an empty ArrayList
+     */
+    @Override
+    public List<Void> snapshotState() {

Review comment:
       Not sure why we would need to store anything here. Could you elaborate on what you consider to be 'active files'?
   From my understanding we cannot reuse previously written `PinotWriterSegment`s, as elements might get sent to different subTasks when recovering from failure.




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r597257012



##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.util.TestLogger;
+import org.apache.pinot.spi.config.table.*;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.HttpWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Base class for PinotSink e2e tests
+ */
+@Testcontainers
+public class PinotTestBase extends TestLogger {
+
+    protected static final Logger LOG = LoggerFactory.getLogger(PinotTestBase.class);
+
+    private static final String DOCKER_IMAGE_NAME = "apachepinot/pinot:0.6.0";
+    private static final Integer PINOT_INTERNAL_BROKER_PORT = 8000;
+    private static final Integer PINOT_INTERNAL_CONTROLLER_PORT = 9000;
+
+    protected static final TableConfig TABLE_CONFIG = PinotTableConfig.getTableConfig();
+    protected static final String TABLE_NAME = TABLE_CONFIG.getTableName();
+    protected static final Schema TABLE_SCHEMA = PinotTableConfig.getTableSchema();
+    protected static PinotTestHelper pinotHelper;
+
+    /**
+     * Creates the Pinot testcontainer. We delay the start of tests until Pinot has started all
+     * internal components. This is identified through a log statement.
+     */
+    @Container
+    public GenericContainer<?> pinot = new GenericContainer<>(DockerImageName.parse(DOCKER_IMAGE_NAME))
+            .withCommand("QuickStart", "-type", "batch")
+            .withExposedPorts(PINOT_INTERNAL_BROKER_PORT, PINOT_INTERNAL_CONTROLLER_PORT)
+            .waitingFor(
+                    // Wait for controller, server and broker instances to be available
+                    new HttpWaitStrategy()
+                            .forPort(PINOT_INTERNAL_CONTROLLER_PORT)
+                            .forPath("/instances")
+                            .forStatusCode(200)
+                            .forResponsePredicate(res -> {
+                                try {
+                                    JsonNode instances = JsonUtils.stringToJsonNode(res).get("instances");
+                                    // Expect 3 instances to be up and running (controller, broker and server)
+                                    return instances.size() == 3;
+                                } catch (IOException e) {
+                                    LOG.error("Error while reading json response in HttpWaitStrategy.", e);
+                                }
+                                return false;
+                            })
+                            // Allow Pinot to take up to 180s for starting up
+                            .withStartupTimeout(Duration.ofSeconds(180))
+            );
+
+    /**
+     * Creates a new instance of the {@link PinotTestHelper} using the testcontainer port mappings
+     * and creates the test table.
+     *
+     * @throws IOException
+     */
+    @BeforeEach
+    public void setUp() throws IOException {
+        pinotHelper = new PinotTestHelper(getPinotHost(), getPinotControllerPort(), getPinotBrokerPort());
+        pinotHelper.createTable(TABLE_CONFIG, TABLE_SCHEMA);
+    }
+
+    /**
+     * Delete the test table after each test.
+     *
+     * @throws Exception
+     */
+    @AfterEach
+    public void tearDown() throws Exception {
+        pinotHelper.deleteTable(TABLE_CONFIG, TABLE_SCHEMA);
+    }
+
+    /**
+     * Returns the host the Pinot container is available at
+     *
+     * @return Pinot container host
+     */
+    protected String getPinotHost() {
+        return this.pinot.getHost();
+    }
+
+
+    /**
+     * Returns the Pinot controller port from the container ports.
+     *
+     * @return Pinot controller port
+     */
+    protected String getPinotControllerPort() {
+        return this.pinot.getMappedPort(PINOT_INTERNAL_CONTROLLER_PORT).toString();
+    }
+
+    /**
+     * Returns the Pinot broker port from the container ports.
+     *
+     * @return Pinot broker port
+     */
+    protected String getPinotBrokerPort() {
+        return this.pinot.getMappedPort(PINOT_INTERNAL_BROKER_PORT).toString();
+    }
+
+    /**
+     * Class defining the elements passed to the {@link PinotSink} during the tests.
+     */
+    protected static class SingleColumnTableRow {
+
+        private String _col1;
+        private Long _timestamp;
+
+        SingleColumnTableRow(@JsonProperty(value = "col1", required = true) String col1,
+                             @JsonProperty(value = "timestamp", required = true) Long timestamp) {
+            this._col1 = col1;
+            this._timestamp = timestamp;
+        }
+
+        @JsonProperty("col1")
+        public String getCol1() {
+            return this._col1;
+        }
+
+        public void setCol1(String _col1) {
+            this._col1 = _col1;
+        }
+
+        @JsonProperty("timestamp")
+        public Long getTimestamp() {
+            return this._timestamp;
+        }
+
+        public void setTimestamp(Long timestamp) {
+            this._timestamp = timestamp;
+        }
+    }
+
+
+    /**
+     * EventTimeExtractor for {@link SingleColumnTableRow} used in e2e tests.
+     * Extracts the timestamp column from {@link SingleColumnTableRow}.
+     */
+    protected static class SingleColumnTableRowEventTimeExtractor extends EventTimeExtractor<SingleColumnTableRow> {

Review comment:
       As the `SingleColumnTableRowEventTimeExtractor` will only be used within sink tests it makes sense to move it there.  




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

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



[GitHub] [bahir-flink] fapaul commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
fapaul commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r597836925



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotSink.java
##########
@@ -0,0 +1,376 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.connector.sink.Committer;
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.api.connector.sink.Sink;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommittable;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.serializer.PinotSinkCommittableSerializer;
+import org.apache.flink.streaming.connectors.pinot.serializer.PinotSinkGlobalCommittableSerializer;
+import org.apache.flink.streaming.connectors.pinot.serializer.PinotSinkWriterStateSerializer;
+import org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter;
+import org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriterState;
+import org.apache.flink.streaming.connectors.pinot.writer.PinotWriterSegment;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Apache Pinot sink that stores objects from upstream Flink tasks in a Apache Pinot table. The sink
+ * can be operated in {@code RuntimeExecutionMode.STREAMING} or {@code RuntimeExecutionMode.BATCH}
+ * mode. But ensure to enable checkpointing when using in streaming mode.
+ *
+ * <p>We advise you to use the provided {@link PinotSink.Builder} to build and configure the
+ * PinotSink. All the communication with the Pinot cluster's table is managed via the Pinot
+ * controller. Thus you need to provide its host and port as well as the target Pinot table.
+ * The {@link TableConfig} and {@link Schema} is automatically retrieved via the Pinot controller API
+ * and therefore does not need to be provided.
+ *
+ * <p>Whenever an element is received by the sink it gets stored in a {@link PinotWriterSegment}. A
+ * {@link PinotWriterSegment} represents exactly one segment that will be pushed to the Pinot
+ * cluster later on. Its size is determined by the customizable {@code maxRowsPerSegment} parameter.
+ * Please note that the maximum segment size that can be handled by this sink is limited by the
+ * lower bound of memory available at each subTask.
+ * Each subTask holds a list of {@link PinotWriterSegment}s of which at most one is active. An
+ * active {@link PinotWriterSegment} is capable of accepting at least one more element. If a
+ * {@link PinotWriterSegment} switches from active to inactive it flushes its
+ * {@code maxRowsPerSegment} elements to disk. The data file is stored in the local filesystem's
+ * temporary directory and contains serialized elements. We use the {@link JsonSerializer} to
+ * serialize elements to JSON.
+ *
+ * <p>On checkpointing all not in-progress {@link PinotWriterSegment}s are transformed into
+ * committables. As the data files need to be shared across nodes, the sink requires access to a
+ * shared filesystem. We use the {@link FileSystemAdapter} for that purpose.
+ * A {@link FileSystemAdapter} is capable of copying a file from the local to the shared filesystem
+ * and vice-versa. A {@link PinotSinkCommittable} contains a reference to a data file on the shared
+ * filesystem as well as the minimum and maximum timestamp contained in the data file. A timestamp -
+ * usually the event time - is extracted from each received element via {@link EventTimeExtractor}.
+ * The timestamps are later on required to follow the guideline for naming Pinot segments.
+ * An eventually existent in-progress {@link PinotWriterSegment}'s state is saved in the snapshot
+ * taken when checkpointing. This ensures that the at-most-once delivery guarantee can be fulfilled
+ * when recovering from failures.
+ *
+ * <p>We use the {@link PinotSinkGlobalCommitter} to collect all created
+ * {@link PinotSinkCommittable}s, create segments from the referenced data files and finally push them
+ * to the Pinot table. Therefore, the minimum and maximum timestamp of all
+ * {@link PinotSinkCommittable} is determined. The segment names are then generated using the
+ * {@link PinotSinkSegmentNameGenerator} which gets the minimum and maximum timestamp as input.
+ * The segment generation starts with downloading the referenced data file from the shared file system
+ * using the provided {@link FileSystemAdapter}. Once this is was completed, we use Pinot's
+ * {@link SegmentIndexCreationDriver} to generate the final segment. Each segment is thereby stored
+ * in a temporary directory on the local filesystem. Next, the segment is uploaded to the Pinot
+ * controller using Pinot's {@link UploadSegmentCommand}.
+ *
+ * <p>To ensure that possible failures are handled accordingly each segment name is checked for
+ * existence within the Pinot cluster before uploading a segment. In case a segment name already
+ * exists, i.e. if the last commit failed partially with some segments already been uploaded, the
+ * existing segment is deleted first. When the elements since the last checkpoint are replayed the
+ * minimum and maximum timestamp of all received elements will be the same. Thus the same set of
+ * segment names is generated and we can delete previous segments by checking for segment name
+ * presence. Note: The {@link PinotSinkSegmentNameGenerator} must be deterministic. We also provide
+ * a {@link SimpleSegmentNameGenerator} which is a simple but for most users suitable segment name
+ * generator.
+ *
+ * <p>Please note that we use the {@link GlobalCommitter} to ensure consistent segment naming. This
+ * comes with performance limitations as a {@link GlobalCommitter} always runs at a parallelism of 1
+ * which results in a clear bottleneck at the {@link PinotSinkGlobalCommitter} that does all the
+ * computational intensive work (i.e. generating and uploading segments). In order to overcome this
+ * issue we introduce a custom multithreading approach within the {@link PinotSinkGlobalCommitter}
+ * to parallelize the segment creation and upload process.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotSink<IN> implements Sink<IN, PinotSinkCommittable, PinotSinkWriterState, PinotSinkGlobalCommittable> {
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final int maxRowsPerSegment;
+    private final String tempDirPrefix;
+    private final JsonSerializer<IN> jsonSerializer;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final FileSystemAdapter fsAdapter;
+    private final EventTimeExtractor<IN> eventTimeExtractor;
+    private final int numCommitThreads;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param maxRowsPerSegment    Maximum number of rows to be stored within a Pinot segment
+     * @param tempDirPrefix        Prefix for temp directories used
+     * @param jsonSerializer       Serializer used to convert elements to JSON
+     * @param eventTimeExtractor   Defines the way event times are extracted from received objects
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Filesystem adapter used to save files for sharing files across nodes
+     * @param numCommitThreads     Number of threads used in the {@link PinotSinkGlobalCommitter} for committing segments
+     */
+    public PinotSink(String pinotControllerHost, String pinotControllerPort, String tableName,

Review comment:
       You should **not** have a public constructor and offer a builder. I'd recommend making the constructor private.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerHttpClient.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+@Internal
+public class PinotControllerHttpClient implements Closeable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerHttpClient.class);
+    protected final String controllerHostPort;
+    protected final CloseableHttpClient httpClient;

Review comment:
       private

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotSinkTest.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.pinot.client.ResultSet;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.opentest4j.AssertionFailedError;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * E2e tests for Pinot Sink using BATCH and STREAMING execution mode
+ */
+public class PinotSinkTest extends PinotTestBase {
+
+    /**
+     * Tests the BATCH execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testBatchSink() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.BATCH);
+        env.setParallelism(2);
+
+        List<SingleColumnTableRow> data = getTestData(12);
+        this.setupDataStream(env, data);
+
+        // Run
+        executeOnMiniCluster(env.getStreamGraph().getJobGraph());
+
+        checkForDataInPinotWithRetry(data, 20);
+    }
+
+    /**
+     * Tests the STREAMING execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testStreamingSink() throws Exception {
+        final Configuration conf = new Configuration();
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(conf);
+        env.setRuntimeMode(RuntimeExecutionMode.STREAMING);
+        env.setParallelism(2);
+        env.enableCheckpointing(50);
+
+        List<SingleColumnTableRow> data = getTestData(12);
+        this.setupDataStream(env, data);
+
+        // Run
+        executeOnMiniCluster(env.getStreamGraph().getJobGraph());
+
+        // We only expect the first 100 elements to be already committed to Pinot.
+        // The remaining would follow once we increase the input data size.
+        // The stream executions stops once the last input tuple was sent to the sink.
+        checkForDataInPinotWithRetry(data, 20);
+    }
+
+    /**
+     * Generates a small test dataset consisting of {@link SingleColumnTableRow}s.
+     *
+     * @return List of SingleColumnTableRow
+     */
+    private List<SingleColumnTableRow> getTestData(int numItems) {
+        return IntStream.range(1, numItems + 1)
+                .mapToObj(num -> "ColValue" + num)
+                .map(col1 -> new SingleColumnTableRow(col1, System.currentTimeMillis()))
+                .collect(Collectors.toList());
+    }
+
+    /**
+     * Executes a given JobGraph on a MiniCluster.
+     *
+     * @param jobGraph JobGraph to execute
+     * @throws Exception
+     */
+    private void executeOnMiniCluster(JobGraph jobGraph) throws Exception {
+        final Configuration config = new Configuration();
+        config.setString(RestOptions.BIND_PORT, "18081-19000");
+        final MiniClusterConfiguration cfg =

Review comment:
       I think it makes more sense to specify the mini cluster with the help of `MiniClusterWithClientResource` and make it a `@ClassRule`. This way the cluster is only instantiated once and it drastically speeds up the tests. https://github.com/apache/flink/blob/fd8bae505281c443d4a74b97a6666845744921e7/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java#L95

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotSinkTest.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.pinot.client.ResultSet;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.opentest4j.AssertionFailedError;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * E2e tests for Pinot Sink using BATCH and STREAMING execution mode
+ */
+public class PinotSinkTest extends PinotTestBase {
+
+    /**
+     * Tests the BATCH execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testBatchSink() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.BATCH);
+        env.setParallelism(2);
+
+        List<SingleColumnTableRow> data = getTestData(12);
+        this.setupDataStream(env, data);
+
+        // Run
+        executeOnMiniCluster(env.getStreamGraph().getJobGraph());
+
+        checkForDataInPinotWithRetry(data, 20);
+    }
+
+    /**
+     * Tests the STREAMING execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testStreamingSink() throws Exception {
+        final Configuration conf = new Configuration();
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(conf);
+        env.setRuntimeMode(RuntimeExecutionMode.STREAMING);
+        env.setParallelism(2);
+        env.enableCheckpointing(50);
+
+        List<SingleColumnTableRow> data = getTestData(12);
+        this.setupDataStream(env, data);
+
+        // Run
+        executeOnMiniCluster(env.getStreamGraph().getJobGraph());
+
+        // We only expect the first 100 elements to be already committed to Pinot.
+        // The remaining would follow once we increase the input data size.
+        // The stream executions stops once the last input tuple was sent to the sink.
+        checkForDataInPinotWithRetry(data, 20);
+    }
+
+    /**
+     * Generates a small test dataset consisting of {@link SingleColumnTableRow}s.
+     *
+     * @return List of SingleColumnTableRow
+     */
+    private List<SingleColumnTableRow> getTestData(int numItems) {
+        return IntStream.range(1, numItems + 1)
+                .mapToObj(num -> "ColValue" + num)
+                .map(col1 -> new SingleColumnTableRow(col1, System.currentTimeMillis()))
+                .collect(Collectors.toList());
+    }
+
+    /**
+     * Executes a given JobGraph on a MiniCluster.
+     *
+     * @param jobGraph JobGraph to execute
+     * @throws Exception
+     */
+    private void executeOnMiniCluster(JobGraph jobGraph) throws Exception {
+        final Configuration config = new Configuration();
+        config.setString(RestOptions.BIND_PORT, "18081-19000");
+        final MiniClusterConfiguration cfg =
+                new MiniClusterConfiguration.Builder()
+                        .setNumTaskManagers(1)
+                        .setNumSlotsPerTaskManager(4)
+                        .setConfiguration(config)
+                        .build();
+
+        try (MiniCluster miniCluster = new MiniCluster(cfg)) {
+            miniCluster.start();
+            miniCluster.executeJobBlocking(jobGraph);
+        }
+    }
+
+    /**
+     * Sets up a DataStream using the provided execution environment and the provided input data.
+     *
+     * @param env  stream execution environment
+     * @param data Input data
+     */
+    private void setupDataStream(StreamExecutionEnvironment env, List<SingleColumnTableRow> data) {
+        // Create test stream
+        DataStream<SingleColumnTableRow> theData =
+                env.fromCollection(data)
+                        .name("Test input");
+
+        String tempDirPrefix = "flink-pinot-connector-test";
+        PinotSinkSegmentNameGenerator segmentNameGenerator = new SimpleSegmentNameGenerator(TABLE_NAME, "flink-connector");
+        FileSystemAdapter fsAdapter = new LocalFileSystemAdapter(tempDirPrefix);
+        JsonSerializer<SingleColumnTableRow> jsonSerializer = new SingleColumnTableRowSerializer();
+
+        EventTimeExtractor<SingleColumnTableRow> eventTimeExtractor = new SingleColumnTableRowEventTimeExtractor();
+
+        PinotSink<SingleColumnTableRow> sink = new PinotSink.Builder<SingleColumnTableRow>(getPinotHost(), getPinotControllerPort(), TABLE_NAME)
+                .withMaxRowsPerSegment(5)
+                .withTempDirectoryPrefix(tempDirPrefix)
+                .withJsonSerializer(jsonSerializer)
+                .withEventTimeExtractor(eventTimeExtractor)
+                .withSegmentNameGenerator(segmentNameGenerator)
+                .withFileSystemAdapter(fsAdapter)
+                .withNumCommitThreads(2)
+                .build();
+
+        // Sink into Pinot
+        theData.sinkTo(sink).name("Pinot sink");
+    }
+
+    /**
+     * As Pinot might take some time to index the recently pushed segments we might need to retry
+     * the {@link #checkForDataInPinot} method multiple times. This method provides a simple wrapper
+     * using linear retry backoff delay.
+     *
+     * @param data                  Data to expect in the Pinot table
+     * @param retryTimeoutInSeconds Maximum duration in seconds to wait for the data to arrive
+     * @throws InterruptedException
+     */
+    private void checkForDataInPinotWithRetry(List<SingleColumnTableRow> data, int retryTimeoutInSeconds) throws InterruptedException {
+        long endTime = System.currentTimeMillis() + 1000L * retryTimeoutInSeconds;
+        // Use max 10 retries with linear retry backoff delay
+        long retryDelay = 1000L / 10 * retryTimeoutInSeconds;
+        do {
+            try {
+                checkForDataInPinot(data);
+                // In case of no error, we can skip further retries
+                return;
+            } catch (AssertionFailedError | PinotControllerApiException e) {

Review comment:
       Do you still allow the `AssertionFailedError` I would assume once the job is finished all data is in pinot.

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.util.TestLogger;
+import org.apache.pinot.spi.config.table.*;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.HttpWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.time.Duration;
+
+/**
+ * Base class for PinotSink e2e tests
+ */
+@Testcontainers
+public class PinotTestBase extends TestLogger {
+
+    protected static final Logger LOG = LoggerFactory.getLogger(PinotTestBase.class);
+
+    private static final String DOCKER_IMAGE_NAME = "apachepinot/pinot:0.6.0";
+    private static final Integer PINOT_INTERNAL_BROKER_PORT = 8000;
+    private static final Integer PINOT_INTERNAL_CONTROLLER_PORT = 9000;
+
+    protected static final TableConfig TABLE_CONFIG = PinotTableConfig.getTableConfig();
+    protected static final String TABLE_NAME = TABLE_CONFIG.getTableName();
+    protected static final Schema TABLE_SCHEMA = PinotTableConfig.getTableSchema();
+    protected static PinotTestHelper pinotHelper;
+
+    /**
+     * Creates the Pinot testcontainer. We delay the start of tests until Pinot has started all
+     * internal components. This is identified through a log statement.
+     */
+    @Container
+    public GenericContainer<?> pinot = new GenericContainer<>(DockerImageName.parse(DOCKER_IMAGE_NAME))

Review comment:
       You should try to make this variable `static` then it is shared between tests otherwise every test starts a new pinot container.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotWriterSegment.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@link PinotWriterSegment} represents exactly one segment that can be found in the Pinot
+ * cluster once the commit has been completed.
+ *
+ * @param <IN> Type of incoming elements
+ */
+@Internal
+public class PinotWriterSegment<IN> implements Serializable {
+
+    private final int maxRowsPerSegment;
+    private final JsonSerializer<IN> jsonSerializer;
+    private final FileSystemAdapter fsAdapter;
+
+    private boolean acceptsElements = true;
+
+    private final List<String> serializedElements;
+    private String dataPathOnSharedFS;
+    private long minTimestamp = Long.MAX_VALUE;
+    private long maxTimestamp = Long.MIN_VALUE;
+
+    /**
+     * @param maxRowsPerSegment Maximum number of rows to be stored within a Pinot segment
+     * @param jsonSerializer    Serializer used to convert elements to JSON
+     * @param fsAdapter         Filesystem adapter used to save files for sharing files across nodes
+     */
+    protected PinotWriterSegment(int maxRowsPerSegment, JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {

Review comment:
       ```suggestion
      PinotWriterSegment(int maxRowsPerSegment, JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
   ```
   package-private?

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import com.google.common.collect.Iterables;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Accepts incoming elements and creates {@link PinotSinkCommittable}s out of them on request.
+ *
+ * @param <IN> Type of incoming elements
+ */
+@Internal
+public class PinotSinkWriter<IN> implements SinkWriter<IN, PinotSinkCommittable, PinotSinkWriterState> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkWriter.class);
+
+    private final int maxRowsPerSegment;
+    private final EventTimeExtractor<IN> eventTimeExtractor;
+    private final JsonSerializer<IN> jsonSerializer;
+
+    private final List<PinotWriterSegment<IN>> activeSegments;
+    private final FileSystemAdapter fsAdapter;
+
+    private final int subtaskId;
+
+    /**
+     * @param subtaskId          Subtask id provided by Flink
+     * @param maxRowsPerSegment  Maximum number of rows to be stored within a Pinot segment
+     * @param eventTimeExtractor Defines the way event times are extracted from received objects
+     * @param jsonSerializer     Serializer used to convert elements to JSON
+     * @param fsAdapter          Filesystem adapter used to save files for sharing files across nodes
+     */
+    public PinotSinkWriter(int subtaskId, int maxRowsPerSegment,
+                           EventTimeExtractor<IN> eventTimeExtractor,
+                           JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        this.subtaskId = subtaskId;
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.eventTimeExtractor = checkNotNull(eventTimeExtractor);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.activeSegments = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements from an upstream tasks and writes them into {@link PinotWriterSegment}
+     *
+     * @param element Object from upstream task
+     * @param context SinkWriter context
+     * @throws IOException
+     */
+    @Override
+    public void write(IN element, Context context) throws IOException {
+        final PinotWriterSegment<IN> inProgressSegment = getOrCreateInProgressSegment();
+        inProgressSegment.write(element, eventTimeExtractor.getEventTime(element, context));
+    }
+
+    /**
+     * Creates {@link PinotSinkCommittable}s from elements previously received via {@link #write}.
+     * If flush is set, all {@link PinotWriterSegment}s are transformed into
+     * {@link PinotSinkCommittable}s. If flush is not set, only currently non-active
+     * {@link PinotSinkCommittable}s are transformed into {@link PinotSinkCommittable}s.
+     * To convert a {@link PinotWriterSegment} into a {@link PinotSinkCommittable} the data gets
+     * written to the shared filesystem. Moreover, minimum and maximum timestamps are identified.
+     * Finally, all {@link PinotWriterSegment}s transformed into {@link PinotSinkCommittable}s are
+     * removed from {@link #activeSegments}.
+     *
+     * @param flush Flush all currently known elements into the {@link PinotSinkCommittable}s
+     * @return List of {@link PinotSinkCommittable} to process in {@link org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter}
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkCommittable> prepareCommit(boolean flush) throws IOException {
+        // Identify segments to commit. If the flush argument is set all segments shall be committed.
+        // Otherwise, take only those PinotWriterSegments that do not accept any more elements.
+        List<PinotWriterSegment<IN>> segmentsToCommit = activeSegments.stream()
+                .filter(s -> flush || !s.acceptsElements())
+                .collect(Collectors.toList());
+        LOG.debug("Identified {} segments to commit [subtaskId={}]", segmentsToCommit.size(), subtaskId);
+
+        LOG.debug("Creating committables... [subtaskId={}]", subtaskId);
+        List<PinotSinkCommittable> committables = new ArrayList<>();
+        for (final PinotWriterSegment<IN> segment : segmentsToCommit) {
+            committables.add(segment.prepareCommit());
+        }
+        LOG.debug("Created {} committables [subtaskId={}]", committables.size(), subtaskId);
+
+        // Remove all PinotWriterSegments that will be emitted within the committables.
+        activeSegments.removeAll(segmentsToCommit);
+        return committables;
+    }
+
+    /**
+     * Gets the {@link PinotWriterSegment} still accepting elements or creates a new one.
+     *
+     * @return {@link PinotWriterSegment} accepting at least one more element
+     */
+    private PinotWriterSegment<IN> getOrCreateInProgressSegment() {
+        final PinotWriterSegment<IN> latestSegment = Iterables.getLast(activeSegments, null);
+        if (latestSegment == null || !latestSegment.acceptsElements()) {
+            final PinotWriterSegment<IN> inProgressSegment = new PinotWriterSegment<>(maxRowsPerSegment, jsonSerializer, fsAdapter);
+            activeSegments.add(inProgressSegment);
+            return inProgressSegment;
+        }
+        return latestSegment;
+    }
+
+    /**
+     * Snapshots the latest PinotWriterSegment (if existent), so that the contained (and not yet
+     * committed) elements can be recovered later on in case of a failure.
+     *
+     * @return A list containing at most one PinotSinkWriterState
+     */
+    @Override
+    public List<PinotSinkWriterState> snapshotState() {
+        final PinotWriterSegment<IN> latestSegment = Iterables.getLast(activeSegments, null);
+        if (latestSegment == null || !latestSegment.acceptsElements()) {
+            return new ArrayList<>();
+        }

Review comment:
       Looks solid 👍 




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594309067



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);

Review comment:
       see above




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

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



[GitHub] [bahir-flink] fapaul commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
fapaul commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r596903804



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerHttpClient.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+@Internal
+public class PinotControllerHttpClient implements Closeable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerHttpClient.class);
+    protected final String controllerHostPort;
+    protected final CloseableHttpClient httpClient;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerHttpClient(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+        httpClient = HttpClients.createDefault();

Review comment:
       `this.`

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotSink.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.connector.sink.Committer;
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.api.connector.sink.Sink;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommittable;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.serializer.PinotSinkCommittableSerializer;
+import org.apache.flink.streaming.connectors.pinot.serializer.PinotSinkGlobalCommittableSerializer;
+import org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter;
+import org.apache.flink.streaming.connectors.pinot.writer.PinotWriterSegment;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Apache Pinot sink that stores objects from upstream Flink tasks in a Apache Pinot table. The sink
+ * can be operated in {@code RuntimeExecutionMode.STREAMING} or {@code RuntimeExecutionMode.BATCH}
+ * mode. But ensure to enable checkpointing when using in streaming mode.
+ *
+ * <p>We advise you to use the provided {@link PinotSink.Builder} to build and configure the
+ * PinotSink. All the communication with the Pinot cluster's table is managed via the Pinot
+ * controller. Thus you need to provide its host and port as well as the target Pinot table.
+ * The {@link TableConfig} and {@link Schema} is automatically retrieved via the Pinot controller API
+ * and therefore does not need to be provided.
+ *
+ * <p>Whenever an element is received by the sink it gets stored in a {@link PinotWriterSegment}. A
+ * {@link PinotWriterSegment} represents exactly one segment that will be pushed to the Pinot
+ * cluster later on. Its size is determined by the customizable {@code maxRowsPerSegment} parameter.
+ * Please note that the maximum segment size that can be handled by this sink is limited by the
+ * lower bound of memory available at each subTask.
+ * Each subTask holds a list of {@link PinotWriterSegment}s of which at most one is active. An
+ * active {@link PinotWriterSegment} is capable of accepting at least one more element. If a
+ * {@link PinotWriterSegment} switches from active to inactive it flushes its
+ * {@code maxRowsPerSegment} elements to disk. The data file is stored in the local filesystem's
+ * temporary directory and contains serialized elements. We use the {@link JsonSerializer} to
+ * serialize elements to JSON.
+ *
+ * <p>On checkpointing all inactive {@link PinotWriterSegment} are transformed into committables. As
+ * the data files need to be shared across nodes, the sink requires access to a shared filesystem. We
+ * use the {@link FileSystemAdapter} for that purpose. A {@link FileSystemAdapter} is capable of
+ * copying a file from the local to the shared filesystem and vice-versa.
+ * A {@link PinotSinkCommittable} contains a reference to a data file on the shared file system as
+ * well as the minimum and maximum timestamp contained in the data file. A timestamp - usually the
+ * event time - is extracted from each received element via {@link EventTimeExtractor}. The
+ * timestamps are later on required to follow the guideline for naming Pinot segments.
+ *
+ * <p>We use the {@link PinotSinkGlobalCommitter} to collect all created
+ * {@link PinotSinkCommittable}s, create segments from the referenced data files and finally push them
+ * to the Pinot table. Therefore, the minimum and maximum timestamp of all
+ * {@link PinotSinkCommittable} is determined. The segment names are then generated using the
+ * {@link PinotSinkSegmentNameGenerator} which gets the minimum and maximum timestamp as input.
+ * The segment generation starts with downloading the referenced data file from the shared file system
+ * using the provided {@link FileSystemAdapter}. Once this is was completed, we use Pinot's
+ * {@link SegmentIndexCreationDriver} to generate the final segment. Each segment is thereby stored
+ * in a temporary directory on the local filesystem. Next, the segment is uploaded to the Pinot
+ * controller using Pinot's {@link UploadSegmentCommand}.
+ *
+ * <p>To ensure that possible failures are handled accordingly each segment name is checked for
+ * existence within the Pinot cluster before uploading a segment. In case a segment name already
+ * exists, i.e. if the last commit failed partially with some segments already been uploaded, the
+ * existing segment is deleted first. When the elements since the last checkpoint are replayed the
+ * minimum and maximum timestamp of all received elements will be the same. Thus the same set of
+ * segment names is generated and we can delete previous segments by checking for segment name
+ * presence. Note: The {@link PinotSinkSegmentNameGenerator} must be deterministic. We also provide
+ * a {@link SimpleSegmentNameGenerator} which is a simple but for most users suitable segment name
+ * generator.
+ *
+ * <p>Please note that we use the {@link GlobalCommitter} to ensure consistent segment naming. This
+ * comes with performance limitations as a {@link GlobalCommitter} always runs at a parallelism of 1
+ * which results in a clear bottleneck at the {@link PinotSinkGlobalCommitter} that does all the
+ * computational intensive work (i.e. generating and uploading segments). In order to overcome this
+ * issue we introduce a custom multithreading approach within the {@link PinotSinkGlobalCommitter}
+ * to parallelize the segment creation and upload process.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotSink<IN> implements Sink<IN, PinotSinkCommittable, Void, PinotSinkGlobalCommittable> {
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final int maxRowsPerSegment;
+    private final String tempDirPrefix;
+    private final JsonSerializer<IN> jsonSerializer;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final FileSystemAdapter fsAdapter;
+    private final EventTimeExtractor<IN> eventTimeExtractor;
+    private final int numCommitThreads;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param maxRowsPerSegment    Maximum number of rows to be stored within a Pinot segment
+     * @param tempDirPrefix        Prefix for temp directories used
+     * @param jsonSerializer       Serializer used to convert elements to JSON
+     * @param eventTimeExtractor   Defines the way event times are extracted from received objects
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Filesystem adapter used to save files for sharing files across nodes
+     * @param numCommitThreads     Number of threads used in the {@link PinotSinkGlobalCommitter} for committing segments
+     */
+    public PinotSink(String pinotControllerHost, String pinotControllerPort, String tableName,
+                     int maxRowsPerSegment, String tempDirPrefix, JsonSerializer<IN> jsonSerializer,
+                     EventTimeExtractor<IN> eventTimeExtractor,
+                     SegmentNameGenerator segmentNameGenerator, FileSystemAdapter fsAdapter,
+                     int numCommitThreads) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+
+        checkArgument(maxRowsPerSegment > 0);
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.eventTimeExtractor = checkNotNull(eventTimeExtractor);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        checkArgument(numCommitThreads > 0);
+        this.numCommitThreads = numCommitThreads;
+    }
+
+    /**
+     * Creates a Pinot sink writer.
+     *
+     * @param context InitContext
+     * @param states  Empty list as the PinotSinkWriter does not accept states.
+     */
+    @Override
+    public PinotSinkWriter<IN> createWriter(InitContext context, List<Void> states) {
+        return new PinotSinkWriter<>(
+                context.getSubtaskId(), maxRowsPerSegment, eventTimeExtractor,
+                jsonSerializer, fsAdapter
+        );
+    }
+
+    /**
+     * The PinotSink does not use a committer. Instead a global committer is used
+     *
+     * @return Empty Optional
+     */
+    @Override
+    public Optional<Committer<PinotSinkCommittable>> createCommitter() {
+        return Optional.empty();
+    }
+
+    /**
+     * Creates the global committer.
+     */
+    @Override
+    public Optional<GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable>> createGlobalCommitter() throws IOException {
+        String timeColumnName = eventTimeExtractor.getTimeColumn();
+        TimeUnit segmentTimeUnit = eventTimeExtractor.getSegmentTimeUnit();
+        PinotSinkGlobalCommitter committer = new PinotSinkGlobalCommitter(
+                pinotControllerHost, pinotControllerPort, tableName, segmentNameGenerator,
+                tempDirPrefix, fsAdapter, timeColumnName, segmentTimeUnit, numCommitThreads
+        );
+        return Optional.of(committer);
+    }
+
+    /**
+     * Creates the committables' serializer.
+     */
+    @Override
+    public Optional<SimpleVersionedSerializer<PinotSinkCommittable>> getCommittableSerializer() {
+        return Optional.of(new PinotSinkCommittableSerializer());
+    }
+
+    /**
+     * Creates the global committables' serializer.
+     */
+    @Override
+    public Optional<SimpleVersionedSerializer<PinotSinkGlobalCommittable>> getGlobalCommittableSerializer() {
+        return Optional.of(new PinotSinkGlobalCommittableSerializer());
+    }
+
+    /**
+     * The PinotSink does not use writer states.
+     *
+     * @return Empty Optional
+     */
+    @Override
+    public Optional<SimpleVersionedSerializer<Void>> getWriterStateSerializer() {
+        return Optional.empty();
+    }
+
+    /**
+     * Builder for configuring a {@link PinotSink}. This is the recommended public API.
+     *
+     * @param <IN> Type of incoming elements
+     */
+    public static class Builder<IN> {
+        String pinotControllerHost;
+        String pinotControllerPort;
+        String tableName;
+        int maxRowsPerSegment;
+        String tempDirPrefix = "flink-connector-pinot";
+        JsonSerializer<IN> jsonSerializer;
+        EventTimeExtractor<IN> eventTimeExtractor;
+        SegmentNameGenerator segmentNameGenerator;
+        FileSystemAdapter fsAdapter;
+        int numCommitThreads = 4;

Review comment:
       Nit: Use a constant variable `static final ...` for the magic number `4` and call it something like `DEFAULT_COMMIT_THREADS`

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,469 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerClient;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemUtils;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+@Internal
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+    private final PinotControllerClient pinotControllerClient;
+    private final File tempDirectory;
+    private final Schema tableSchema;
+    private final TableConfig tableConfig;
+    private final ExecutorService pool;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param tempDirPrefix        Prefix for directory to store temporary files in
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     * @param numCommitThreads     Number of threads used to commit the committables
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort,
+                                    String tableName, SegmentNameGenerator segmentNameGenerator,
+                                    String tempDirPrefix, FileSystemAdapter fsAdapter,
+                                    String timeColumnName, TimeUnit segmentTimeUnit,
+                                    int numCommitThreads) throws IOException {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+        pinotControllerClient = new PinotControllerClient(pinotControllerHost, pinotControllerPort);
+
+        // Create directory that temporary files will be stored in
+        tempDirectory = Files.createTempDirectory(tempDirPrefix).toFile();
+
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        tableSchema = pinotControllerClient.getSchema(tableName);
+        tableConfig = pinotControllerClient.getTableConfig(tableName);
+
+        // We use a thread pool in order to parallelize the segment creation and segment upload
+        checkArgument(numCommitThreads > 0);
+        pool = Executors.newFixedThreadPool(numCommitThreads);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Holds identified global committables whose commit needs to be retried
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                pinotControllerClient.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.debug("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+            // Commit all segments in globalCommittable
+            for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+                String dataFilePath = globalCommittable.getDataFilePaths().get(sequenceId);
+                // Get segment names with increasing sequenceIds
+                String segmentName = getSegmentName(globalCommittable, sequenceId);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        pinotControllerHost, pinotControllerPort, tempDirectory, fsAdapter,
+                        dataFilePath, segmentName, tableSchema, tableConfig, timeColumnName,
+                        segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            boolean commitSucceeded = true;
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        commitSucceeded = false;
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (InterruptedException | ExecutionException e) {
+                // In case of an exception thrown while accessing commit status, mark the whole
+                // globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error("Accessing a SegmentCommitter thread errored with {}", e.getMessage(), e);
+            }
+
+            if (commitSucceeded) {
+                // If commit succeeded, cleanup the data files stored on the shared file system. In
+                // case the commit of at least one of the segments failed, nothing will be cleaned
+                // up here to enable retrying failed commits (data files must therefore stay
+                // available on the shared filesystem).
+                for (String path : globalCommittable.getDataFilePaths()) {
+                    fsAdapter.deleteFromSharedFileSystem(path);
+                }
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Closes the Pinot controller http client, clears the created temporary directory and
+     * shuts the thread pool down.
+     */
+    @Override
+    public void close() throws IOException {
+        pinotControllerClient.close();
+        tempDirectory.delete();
+        pool.shutdown();
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return segmentNameGenerator.generateSegmentName(sequenceId,
+                globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = getSegmentName(globalCommittable, sequenceId);
+            if (pinotControllerClient.tableHasSegment(tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    private static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    private static class SegmentCommitter implements Callable<Boolean> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class);
+
+        private final String pinotControllerHost;
+        private final String pinotControllerPort;
+        private final File tempDirectory;
+        private final FileSystemAdapter fsAdapter;
+        private final String dataFilePath;
+        private final String segmentName;
+        private final Schema tableSchema;
+        private final TableConfig tableConfig;
+        private final String timeColumnName;
+        private final TimeUnit segmentTimeUnit;
+
+        /**
+         * @param pinotControllerHost Host of the Pinot controller
+         * @param pinotControllerPort Port of the Pinot controller
+         * @param tempDirectory       Directory to store temporary files in
+         * @param fsAdapter           Filesystem adapter used to load data files from the shared file system
+         * @param dataFilePath        Data file to load from the shared file system
+         * @param segmentName         Name of the segment to create and commit
+         * @param tableSchema         Pinot table schema
+         * @param tableConfig         Pinot table config
+         * @param timeColumnName      Name of the column containing the timestamp
+         * @param segmentTimeUnit     Unit of the time column
+         */
+        SegmentCommitter(String pinotControllerHost, String pinotControllerPort,
+                         File tempDirectory, FileSystemAdapter fsAdapter,
+                         String dataFilePath, String segmentName, Schema tableSchema,
+                         TableConfig tableConfig, String timeColumnName,
+                         TimeUnit segmentTimeUnit) {
+            this.pinotControllerHost = pinotControllerHost;
+            this.pinotControllerPort = pinotControllerPort;
+            this.tempDirectory = tempDirectory;
+            this.fsAdapter = fsAdapter;
+            this.dataFilePath = dataFilePath;
+            this.segmentName = segmentName;
+            this.tableSchema = tableSchema;
+            this.tableConfig = tableConfig;
+            this.timeColumnName = timeColumnName;
+            this.segmentTimeUnit = segmentTimeUnit;
+        }
+
+        /**
+         * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment
+         * and finally uploads the segment to the Pinot controller
+         *
+         * @return True if the commit succeeded
+         */
+        @Override
+        public Boolean call() {
+            // Local copy of data file stored on the shared filesystem
+            File segmentData = null;
+            // File containing the final Pinot segment
+            File segmentFile = null;
+            try {
+                // Download data file from the shared filesystem
+                LOG.debug("Downloading data file {} from shared file system...", dataFilePath);
+                List<String> serializedElements = fsAdapter.readFromSharedFileSystem(dataFilePath);
+                segmentData = FileSystemUtils.writeToLocalFile(serializedElements, tempDirectory);
+                LOG.debug("Successfully downloaded data file {} from shared file system", dataFilePath);
+
+                segmentFile = FileSystemUtils.createFileInDir(tempDirectory);
+                LOG.debug("Creating segment in " + segmentFile.getAbsolutePath());
+
+                // Creates a segment with name `segmentName` in `segmentFile`
+                generateSegment(segmentData, segmentFile, true);
+
+                // Uploads the recently created segment to the Pinot controller
+                uploadSegment(segmentFile);
+
+                // Commit successful
+                return true;
+            } catch (IOException e) {
+                LOG.error("Error while committing segment data stored on shared filesystem.", e);
+
+                // Commit failed
+                return false;
+            } finally {
+                // Finally cleanup all files created on the local filesystem
+                if (segmentData != null) {
+                    segmentData.delete();
+                }
+                if (segmentFile != null) {
+                    segmentFile.delete();
+                }
+            }
+        }
+
+        /**
+         * Creates a segment from the given parameters.
+         * This method was adapted from {@link org.apache.pinot.tools.admin.command.CreateSegmentCommand}.
+         *
+         * @param dataFile                  File containing the JSON data
+         * @param outDir                    Segment target path
+         * @param _postCreationVerification Verify segment after generation
+         */
+        private void generateSegment(File dataFile, File outDir, Boolean _postCreationVerification) {
+            SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(tableConfig, tableSchema);
+            segmentGeneratorConfig.setSegmentName(segmentName);
+            segmentGeneratorConfig.setSegmentTimeUnit(segmentTimeUnit);
+            segmentGeneratorConfig.setTimeColumnName(timeColumnName);
+            segmentGeneratorConfig.setInputFilePath(dataFile.getPath());
+            segmentGeneratorConfig.setFormat(FileFormat.JSON);
+            segmentGeneratorConfig.setOutDir(outDir.getPath());
+            segmentGeneratorConfig.setTableName(tableConfig.getTableName());
+
+            try {
+                SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+                driver.init(segmentGeneratorConfig);
+                driver.build();
+                File indexDir = new File(outDir, segmentName);
+                LOG.debug("Successfully created segment: {} in directory: {}", segmentName, indexDir);
+                if (_postCreationVerification) {
+                    LOG.debug("Verifying the segment by loading it");
+                    ImmutableSegment segment = ImmutableSegmentLoader.load(indexDir, ReadMode.mmap);
+                    LOG.debug("Successfully loaded segment: {} of size: {} bytes", segmentName,
+                            segment.getSegmentSizeBytes());
+                    segment.destroy();
+                }
+            } catch (Exception e) {
+                // SegmentIndexCreationDriverImpl throws generic Exceptions during init and build
+                // ImmutableSegmentLoader throws generic Exception during load

Review comment:
       Nit: move this comment above the exception.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);

Review comment:
       Ah, I forgot about it...

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.util.TestLogger;
+import org.apache.pinot.spi.config.table.*;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.HttpWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Base class for PinotSink e2e tests
+ */
+@Testcontainers
+public class PinotTestBase extends TestLogger {
+
+    protected static final Logger LOG = LoggerFactory.getLogger(PinotTestBase.class);
+
+    private static final String DOCKER_IMAGE_NAME = "apachepinot/pinot:0.6.0";
+    private static final Integer PINOT_INTERNAL_BROKER_PORT = 8000;
+    private static final Integer PINOT_INTERNAL_CONTROLLER_PORT = 9000;
+
+    protected static final TableConfig TABLE_CONFIG = PinotTableConfig.getTableConfig();
+    protected static final String TABLE_NAME = TABLE_CONFIG.getTableName();
+    protected static final Schema TABLE_SCHEMA = PinotTableConfig.getTableSchema();
+    protected static PinotTestHelper pinotHelper;
+
+    /**
+     * Creates the Pinot testcontainer. We delay the start of tests until Pinot has started all
+     * internal components. This is identified through a log statement.
+     */
+    @Container
+    public GenericContainer<?> pinot = new GenericContainer<>(DockerImageName.parse(DOCKER_IMAGE_NAME))
+            .withCommand("QuickStart", "-type", "batch")
+            .withExposedPorts(PINOT_INTERNAL_BROKER_PORT, PINOT_INTERNAL_CONTROLLER_PORT)
+            .waitingFor(
+                    // Wait for controller, server and broker instances to be available
+                    new HttpWaitStrategy()
+                            .forPort(PINOT_INTERNAL_CONTROLLER_PORT)
+                            .forPath("/instances")
+                            .forStatusCode(200)
+                            .forResponsePredicate(res -> {
+                                try {
+                                    JsonNode instances = JsonUtils.stringToJsonNode(res).get("instances");
+                                    // Expect 3 instances to be up and running (controller, broker and server)
+                                    return instances.size() == 3;
+                                } catch (IOException e) {
+                                    LOG.error("Error while reading json response in HttpWaitStrategy.", e);
+                                }
+                                return false;
+                            })
+                            // Allow Pinot to take up to 180s for starting up
+                            .withStartupTimeout(Duration.ofSeconds(180))
+            );
+
+    /**
+     * Creates a new instance of the {@link PinotTestHelper} using the testcontainer port mappings
+     * and creates the test table.
+     *
+     * @throws IOException
+     */
+    @BeforeEach
+    public void setUp() throws IOException {
+        pinotHelper = new PinotTestHelper(getPinotHost(), getPinotControllerPort(), getPinotBrokerPort());
+        pinotHelper.createTable(TABLE_CONFIG, TABLE_SCHEMA);
+    }
+
+    /**
+     * Delete the test table after each test.
+     *
+     * @throws Exception
+     */
+    @AfterEach
+    public void tearDown() throws Exception {
+        pinotHelper.deleteTable(TABLE_CONFIG, TABLE_SCHEMA);
+    }
+
+    /**
+     * Returns the host the Pinot container is available at
+     *
+     * @return Pinot container host
+     */
+    protected String getPinotHost() {
+        return this.pinot.getHost();
+    }
+
+
+    /**
+     * Returns the Pinot controller port from the container ports.
+     *
+     * @return Pinot controller port
+     */
+    protected String getPinotControllerPort() {
+        return this.pinot.getMappedPort(PINOT_INTERNAL_CONTROLLER_PORT).toString();
+    }
+
+    /**
+     * Returns the Pinot broker port from the container ports.
+     *
+     * @return Pinot broker port
+     */
+    protected String getPinotBrokerPort() {

Review comment:
       private?

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotSinkTest.java
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.pinot.client.ResultSet;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.opentest4j.AssertionFailedError;
+
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * E2e tests for Pinot Sink using BATCH and STREAMING execution mode
+ */
+public class PinotSinkTest extends PinotTestBase {
+
+    /**
+     * Tests the BATCH execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testBatchSink() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.BATCH);
+        env.setParallelism(2);
+
+        List<SingleColumnTableRow> data = getTestData(12);
+        this.setupDataStream(env, data);
+
+        // Run
+        env.execute();
+
+        checkForDataInPinotWithRetry(data, data.size(), 20);

Review comment:
       Probably the same question again why not all records are present.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotSink.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.connector.sink.Committer;
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.api.connector.sink.Sink;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommittable;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.serializer.PinotSinkCommittableSerializer;
+import org.apache.flink.streaming.connectors.pinot.serializer.PinotSinkGlobalCommittableSerializer;
+import org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter;
+import org.apache.flink.streaming.connectors.pinot.writer.PinotWriterSegment;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Apache Pinot sink that stores objects from upstream Flink tasks in a Apache Pinot table. The sink
+ * can be operated in {@code RuntimeExecutionMode.STREAMING} or {@code RuntimeExecutionMode.BATCH}
+ * mode. But ensure to enable checkpointing when using in streaming mode.
+ *
+ * <p>We advise you to use the provided {@link PinotSink.Builder} to build and configure the
+ * PinotSink. All the communication with the Pinot cluster's table is managed via the Pinot
+ * controller. Thus you need to provide its host and port as well as the target Pinot table.
+ * The {@link TableConfig} and {@link Schema} is automatically retrieved via the Pinot controller API
+ * and therefore does not need to be provided.
+ *
+ * <p>Whenever an element is received by the sink it gets stored in a {@link PinotWriterSegment}. A
+ * {@link PinotWriterSegment} represents exactly one segment that will be pushed to the Pinot
+ * cluster later on. Its size is determined by the customizable {@code maxRowsPerSegment} parameter.
+ * Please note that the maximum segment size that can be handled by this sink is limited by the
+ * lower bound of memory available at each subTask.
+ * Each subTask holds a list of {@link PinotWriterSegment}s of which at most one is active. An
+ * active {@link PinotWriterSegment} is capable of accepting at least one more element. If a
+ * {@link PinotWriterSegment} switches from active to inactive it flushes its
+ * {@code maxRowsPerSegment} elements to disk. The data file is stored in the local filesystem's
+ * temporary directory and contains serialized elements. We use the {@link JsonSerializer} to
+ * serialize elements to JSON.
+ *
+ * <p>On checkpointing all inactive {@link PinotWriterSegment} are transformed into committables. As

Review comment:
       You also have to deal with the active segments :) Once this is implemented update the doc string

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,469 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerClient;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemUtils;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+@Internal
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+    private final PinotControllerClient pinotControllerClient;
+    private final File tempDirectory;
+    private final Schema tableSchema;
+    private final TableConfig tableConfig;
+    private final ExecutorService pool;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param tempDirPrefix        Prefix for directory to store temporary files in
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     * @param numCommitThreads     Number of threads used to commit the committables
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort,
+                                    String tableName, SegmentNameGenerator segmentNameGenerator,
+                                    String tempDirPrefix, FileSystemAdapter fsAdapter,
+                                    String timeColumnName, TimeUnit segmentTimeUnit,
+                                    int numCommitThreads) throws IOException {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+        pinotControllerClient = new PinotControllerClient(pinotControllerHost, pinotControllerPort);
+
+        // Create directory that temporary files will be stored in
+        tempDirectory = Files.createTempDirectory(tempDirPrefix).toFile();
+
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        tableSchema = pinotControllerClient.getSchema(tableName);
+        tableConfig = pinotControllerClient.getTableConfig(tableName);
+
+        // We use a thread pool in order to parallelize the segment creation and segment upload
+        checkArgument(numCommitThreads > 0);
+        pool = Executors.newFixedThreadPool(numCommitThreads);

Review comment:
       `this.`

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/external/EventTimeExtractor.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.flink.streaming.connectors.pinot.external;
+
+import org.apache.flink.api.connector.sink.SinkWriter;
+
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Defines the interface for event time extractors
+ *
+ * @param <IN> Type of incoming elements
+ */
+public abstract class EventTimeExtractor<IN> implements Serializable {

Review comment:
       interface?

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotSinkTest.java
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.pinot.client.ResultSet;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.opentest4j.AssertionFailedError;
+
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * E2e tests for Pinot Sink using BATCH and STREAMING execution mode
+ */
+public class PinotSinkTest extends PinotTestBase {
+
+    /**
+     * Tests the BATCH execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testBatchSink() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.BATCH);
+        env.setParallelism(2);
+
+        List<SingleColumnTableRow> data = getTestData(12);
+        this.setupDataStream(env, data);
+
+        // Run
+        env.execute();
+
+        checkForDataInPinotWithRetry(data, data.size(), 20);
+    }
+
+    /**
+     * Tests the STREAMING execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testStreamingSink() throws Exception {
+        final Configuration conf = new Configuration();
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(conf);
+        env.setRuntimeMode(RuntimeExecutionMode.STREAMING);
+        env.setParallelism(2);
+        env.enableCheckpointing(50);
+
+        List<SingleColumnTableRow> data = getTestData(1000);
+        this.setupDataStream(env, data);
+
+        // Run
+        env.execute();
+
+        // We only expect the first 100 elements to be already committed to Pinot.
+        // The remaining would follow once we increase the input data size.
+        // The stream executions stops once the last input tuple was sent to the sink.
+        checkForDataInPinotWithRetry(data, 100, 20);

Review comment:
       Why do we not get all records? Since we use `env.execute()` which blocks until the job finishes I would expect all records in pinot.

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.util.TestLogger;
+import org.apache.pinot.spi.config.table.*;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.HttpWaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Base class for PinotSink e2e tests
+ */
+@Testcontainers
+public class PinotTestBase extends TestLogger {
+
+    protected static final Logger LOG = LoggerFactory.getLogger(PinotTestBase.class);
+
+    private static final String DOCKER_IMAGE_NAME = "apachepinot/pinot:0.6.0";
+    private static final Integer PINOT_INTERNAL_BROKER_PORT = 8000;
+    private static final Integer PINOT_INTERNAL_CONTROLLER_PORT = 9000;
+
+    protected static final TableConfig TABLE_CONFIG = PinotTableConfig.getTableConfig();
+    protected static final String TABLE_NAME = TABLE_CONFIG.getTableName();
+    protected static final Schema TABLE_SCHEMA = PinotTableConfig.getTableSchema();
+    protected static PinotTestHelper pinotHelper;
+
+    /**
+     * Creates the Pinot testcontainer. We delay the start of tests until Pinot has started all
+     * internal components. This is identified through a log statement.
+     */
+    @Container
+    public GenericContainer<?> pinot = new GenericContainer<>(DockerImageName.parse(DOCKER_IMAGE_NAME))
+            .withCommand("QuickStart", "-type", "batch")
+            .withExposedPorts(PINOT_INTERNAL_BROKER_PORT, PINOT_INTERNAL_CONTROLLER_PORT)
+            .waitingFor(
+                    // Wait for controller, server and broker instances to be available
+                    new HttpWaitStrategy()
+                            .forPort(PINOT_INTERNAL_CONTROLLER_PORT)
+                            .forPath("/instances")
+                            .forStatusCode(200)
+                            .forResponsePredicate(res -> {
+                                try {
+                                    JsonNode instances = JsonUtils.stringToJsonNode(res).get("instances");
+                                    // Expect 3 instances to be up and running (controller, broker and server)
+                                    return instances.size() == 3;
+                                } catch (IOException e) {
+                                    LOG.error("Error while reading json response in HttpWaitStrategy.", e);
+                                }
+                                return false;
+                            })
+                            // Allow Pinot to take up to 180s for starting up
+                            .withStartupTimeout(Duration.ofSeconds(180))
+            );
+
+    /**
+     * Creates a new instance of the {@link PinotTestHelper} using the testcontainer port mappings
+     * and creates the test table.
+     *
+     * @throws IOException
+     */
+    @BeforeEach
+    public void setUp() throws IOException {
+        pinotHelper = new PinotTestHelper(getPinotHost(), getPinotControllerPort(), getPinotBrokerPort());
+        pinotHelper.createTable(TABLE_CONFIG, TABLE_SCHEMA);
+    }
+
+    /**
+     * Delete the test table after each test.
+     *
+     * @throws Exception
+     */
+    @AfterEach
+    public void tearDown() throws Exception {
+        pinotHelper.deleteTable(TABLE_CONFIG, TABLE_SCHEMA);
+    }
+
+    /**
+     * Returns the host the Pinot container is available at
+     *
+     * @return Pinot container host
+     */
+    protected String getPinotHost() {
+        return this.pinot.getHost();
+    }
+
+
+    /**
+     * Returns the Pinot controller port from the container ports.
+     *
+     * @return Pinot controller port
+     */
+    protected String getPinotControllerPort() {
+        return this.pinot.getMappedPort(PINOT_INTERNAL_CONTROLLER_PORT).toString();
+    }
+
+    /**
+     * Returns the Pinot broker port from the container ports.
+     *
+     * @return Pinot broker port
+     */
+    protected String getPinotBrokerPort() {
+        return this.pinot.getMappedPort(PINOT_INTERNAL_BROKER_PORT).toString();
+    }
+
+    /**
+     * Class defining the elements passed to the {@link PinotSink} during the tests.
+     */
+    protected static class SingleColumnTableRow {
+
+        private String _col1;
+        private Long _timestamp;
+
+        SingleColumnTableRow(@JsonProperty(value = "col1", required = true) String col1,
+                             @JsonProperty(value = "timestamp", required = true) Long timestamp) {
+            this._col1 = col1;
+            this._timestamp = timestamp;
+        }
+
+        @JsonProperty("col1")
+        public String getCol1() {
+            return this._col1;
+        }
+
+        public void setCol1(String _col1) {
+            this._col1 = _col1;
+        }
+
+        @JsonProperty("timestamp")
+        public Long getTimestamp() {
+            return this._timestamp;
+        }
+
+        public void setTimestamp(Long timestamp) {
+            this._timestamp = timestamp;
+        }
+    }
+
+
+    /**
+     * EventTimeExtractor for {@link SingleColumnTableRow} used in e2e tests.
+     * Extracts the timestamp column from {@link SingleColumnTableRow}.
+     */
+    protected static class SingleColumnTableRowEventTimeExtractor extends EventTimeExtractor<SingleColumnTableRow> {

Review comment:
       Move to `PinotSinkTest` because it is only used there?




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594304434



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import com.google.common.collect.Iterables;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Accepts incoming elements and creates {@link PinotSinkCommittable}s out of them on request.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotSinkWriter<IN> implements SinkWriter<IN, PinotSinkCommittable, Void> {
+
+    private static final Logger LOG = LoggerFactory.getLogger("PinotSinkWriter");
+
+    private final int maxRowsPerSegment;
+    private EventTimeExtractor<IN> eventTimeExtractor;
+    private final String tempDirPrefix;
+    private final JsonSerializer<IN> jsonSerializer;
+
+    private final List<PinotWriterSegment<IN>> activeSegments;
+    private final FileSystemAdapter fsAdapter;
+
+    private final int subtaskId;
+
+    /**
+     * @param subtaskId          Subtask id provided by Flink
+     * @param maxRowsPerSegment  Maximum number of rows to be stored within a Pinot segment
+     * @param eventTimeExtractor Defines the way event times are extracted from received objects
+     * @param tempDirPrefix      Prefix for temp directories used
+     * @param jsonSerializer     Serializer used to convert elements to JSON
+     * @param fsAdapter          Filesystem adapter used to save files for sharing files across nodes
+     */
+    public PinotSinkWriter(int subtaskId, int maxRowsPerSegment, EventTimeExtractor<IN> eventTimeExtractor, String tempDirPrefix, JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        this.subtaskId = subtaskId;
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.eventTimeExtractor = checkNotNull(eventTimeExtractor);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.activeSegments = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements from an upstream tasks and writes them into {@link PinotWriterSegment}
+     *
+     * @param element Object from upstream task
+     * @param context SinkWriter context
+     * @throws IOException
+     */
+    @Override
+    public void write(IN element, Context context) throws IOException {
+        final PinotWriterSegment<IN> inProgressSegment = this.getOrCreateInProgressSegment();
+        inProgressSegment.write(element, this.eventTimeExtractor.getEventTime(element, context));
+    }
+
+    /**
+     * Creates {@link PinotSinkCommittable}s from elements received via {@link #write}
+     *
+     * @param flush Flush all currently known elements into the {@link PinotSinkCommittable}s
+     * @return List of {@link PinotSinkCommittable} to process in {@link org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter}
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkCommittable> prepareCommit(boolean flush) throws IOException {

Review comment:
       now using debug instead of info all over the place




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594303464



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkCommittableSerializer.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.flink.streaming.connectors.pinot.serializer;
+
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+
+/**
+ * Serializer for {@link PinotSinkCommittable}
+ */
+public class PinotSinkCommittableSerializer implements SimpleVersionedSerializer<PinotSinkCommittable> {
+
+    private static final int CURRENT_VERSION = 0;
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(PinotSinkCommittable pinotSinkCommittable) {
+        return SerializationUtils.serialize(pinotSinkCommittable);
+    }
+
+    @Override
+    public PinotSinkCommittable deserialize(int i, byte[] bytes) {

Review comment:
       updated to use custom serializer & deserializer with version checking

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkGlobalCommittableSerializer.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.flink.streaming.connectors.pinot.serializer;
+
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommittable;
+
+/**
+ * Serializer for {@link PinotSinkGlobalCommittable}
+ */
+public class PinotSinkGlobalCommittableSerializer implements SimpleVersionedSerializer<PinotSinkGlobalCommittable> {

Review comment:
       updated to use custom serializer & deserializer with version checking




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r592800143



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotWriterSegment.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@link PinotWriterSegment} represents exactly one segment that can be found in the Pinot
+ * cluster once the commit has been completed.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotWriterSegment<IN> implements Serializable {
+
+    private static final Logger LOG = LoggerFactory.getLogger("PinotWriterSegment");
+
+    private final int maxRowsPerSegment;
+    private final String tempDirPrefix;
+    private final JsonSerializer<IN> jsonSerializer;
+    private final FileSystemAdapter fsAdapter;
+
+    private boolean acceptsElements = true;
+
+    private final List<IN> elements;
+    private File dataFile;
+    private long minTimestamp = Long.MAX_VALUE;
+    private long maxTimestamp = Long.MIN_VALUE;
+
+    /**
+     * @param maxRowsPerSegment Maximum number of rows to be stored within a Pinot segment
+     * @param tempDirPrefix     Prefix for temp directories used
+     * @param jsonSerializer    Serializer used to convert elements to JSON
+     * @param fsAdapter         Filesystem adapter used to save files for sharing files across nodes
+     */
+    protected PinotWriterSegment(int maxRowsPerSegment, String tempDirPrefix, JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        checkArgument(maxRowsPerSegment > 0L);
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.elements = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements and stores them in memory until either {@link #maxRowsPerSegment} is reached
+     * or {@link #prepareCommit} is called.
+     *
+     * @param element   Object from upstream task
+     * @param timestamp Timestamp assigned to element
+     * @throws IOException
+     */
+    public void write(IN element, long timestamp) throws IOException {
+        if (!this.acceptsElements()) {
+            throw new IllegalStateException("This PinotSegmentWriter does not accept any elements anymore.");
+        }
+        this.elements.add(element);
+        this.minTimestamp = Long.min(this.minTimestamp, timestamp);
+        this.maxTimestamp = Long.max(this.maxTimestamp, timestamp);
+
+        // Writes elements to local filesystem once the maximum number of items is reached
+        if (this.elements.size() == this.maxRowsPerSegment) {

Review comment:
       The current version of the `FileSystemAdapter` only supports copying local files to the shared FS and vice versa which should allow maximum flexibility when it comes to extensibility.
   I nonetheless see your point that we can avoid unnecessary I/O here. Therefore, I'd suggest to change the `FileSystemAdapter`'s API to accept a list of serialized elements to write to the shared filesystem. Those adapters that need a local file to upload could first write to local disk before uploading and all others could directly write to the respective filesystem.




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594702631



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);

Review comment:
       I guess supporting changing schemas over time is out-of-scope for this PR, as it would require versioning of tuples etc.
   The schema and tableConfig retrieval is now handled in the constructor.




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594695951



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.info("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {
+        HttpGet httpget = new HttpGet(this.controllerHostPort + path);
+        LOG.info("Sending GET request to {}", path);
+        return this.execute(httpget);
+    }
+
+    /**
+     * Issues a DELETE request to the Pinot controller API.
+     *
+     * @param path Path to issue DELETE request to
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse delete(String path) throws IOException {
+        HttpDelete httpdelete = new HttpDelete(this.controllerHostPort + path);
+        LOG.info("Sending DELETE request to {}", path);
+        return this.execute(httpdelete);
+    }
+
+    /**
+     * Checks whether the provided segment name is registered with the given table.
+     *
+     * @param tableName   Target table's name
+     * @param segmentName Segment name to check
+     * @return True if segment with the provided name exists
+     * @throws IOException
+     */
+    public boolean tableHasSegment(String tableName, String segmentName) throws IOException {
+        ApiResponse res = this.get(String.format("/tables/%s/%s/metadata", tableName, segmentName));
+
+        if (res.statusLine.getStatusCode() == 200) {
+            // A segment named `segmentName` exists within the table named `tableName`
+            return true;
+        }
+        if (res.statusLine.getStatusCode() == 404) {
+            // There is no such segment named `segmentName` within the table named `tableName`
+            // (or the table named `tableName` does not exist)
+            return false;
+        }
+
+        // Received an unexpected status code
+        throw new PinotControllerApiException(res.responseBody);
+    }
+
+    /**
+     * Deletes a segment from a table.
+     *
+     * @param tableName   Target table's name
+     * @param segmentName Identifies the segment to delete
+     * @throws IOException
+     */
+    public void deleteSegment(String tableName, String segmentName) throws IOException {
+        ApiResponse res = this.delete(String.format("/tables/%s/%s", tableName, segmentName));
+
+        if (res.statusLine.getStatusCode() != 200) {
+            LOG.error("Could not delete segment {} from table {}. Pinot controller returned: {}", tableName, segmentName, res.responseBody);
+            throw new PinotControllerApiException(res.responseBody);
+        }
+    }
+
+    /**
+     * Fetches a Pinot table's schema via the Pinot controller API.
+     *
+     * @param tableName Target table's name
+     * @return Pinot table schema
+     * @throws IOException
+     */
+    public Schema getSchema(String tableName) throws IOException {
+        Schema schema;
+        ApiResponse res = this.get(String.format("/tables/%s/schema", tableName));
+        LOG.info("Get schema request for table {} returned {}", tableName, res.responseBody);
+
+        if (res.statusLine.getStatusCode() != 200) {
+            throw new PinotControllerApiException(res.responseBody);
+        }
+
+        try {
+            schema = JsonUtils.stringToObject(res.responseBody, Schema.class);
+        } catch (Exception e) {
+            throw new IllegalStateException("Caught exception while reading schema from Pinot Controller's response: " + res.responseBody, e);
+        }
+        LOG.info("Retrieved schema: {}", schema.toSingleLineJsonString());
+        return schema;
+    }
+
+    /**
+     * Fetches a Pinot table's configuration via the Pinot controller API.
+     *
+     * @param tableName Target table's name
+     * @return Pinot table configuration
+     * @throws IOException
+     */
+    public TableConfig getTableConfig(String tableName) throws IOException {
+        TableConfig tableConfig;
+        ApiResponse res = this.get(String.format("/tables/%s", tableName));
+        LOG.info("Get table config request for table {} returned {}", tableName, res.responseBody);
+
+        try {
+            String tableConfigAsJson = JsonUtils.stringToJsonNode(res.responseBody).get("OFFLINE").toString();
+            tableConfig = JsonUtils.stringToObject(tableConfigAsJson, TableConfig.class);
+        } catch (Exception e) {
+            throw new IllegalStateException("Caught exception while reading table config from Pinot Controller's response: " + res.responseBody, e);

Review comment:
       You're right, I've replaced `IllegalStateException` with `PinotControllerApiException`




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r598360934



##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotSinkTest.java
##########
@@ -0,0 +1,396 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.connector.sink.SinkWriter;
+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.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.flink.util.Preconditions;
+import org.apache.pinot.client.PinotClientException;
+import org.apache.pinot.client.ResultSet;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.opentest4j.AssertionFailedError;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * E2e tests for Pinot Sink using BATCH and STREAMING execution mode
+ */
+public class PinotSinkTest extends PinotTestBase {
+
+    private static final int MAX_ROWS_PER_SEGMENT = 5;
+    private static final long STREAMING_CHECKPOINTING_INTERVAL = 50;
+    private static final int DATA_CHECKING_TIMEOUT_SECONDS = 60;
+    private static final AtomicBoolean hasFailedOnce = new AtomicBoolean(false);
+    private static CountDownLatch latch;
+
+    @BeforeEach
+    public void setUp() throws IOException {
+        super.setUp();
+        // Reset hasFailedOnce flag used during failure recovery testing before each test.
+        hasFailedOnce.set(false);
+        // Reset latch used to keep the generator streaming source up until the test is completed.
+        latch = new CountDownLatch(1);
+    }
+
+    /**
+     * Tests the BATCH execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testBatchSink() throws Exception {
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.BATCH);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+        env.setParallelism(2);
+
+        List<String> rawData = getRawTestData(12);
+        DataStream<SingleColumnTableRow> dataStream = setupBatchDataSource(env, rawData);
+        setupSink(dataStream);
+
+        // Run
+        env.execute();
+
+        // Check for data in Pinot
+        checkForDataInPinotWithRetry(rawData);
+    }
+
+    /**
+     * Tests failure recovery of the {@link PinotSink} using BATCH execution mode.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testFailureRecoveryInBatchingSink() throws Exception {
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.BATCH);
+        env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 10));
+        env.setParallelism(2);
+
+        List<String> rawData = getRawTestData(12);
+        DataStream<SingleColumnTableRow> dataStream = setupBatchDataSource(env, rawData);
+        dataStream = setupFailingMapper(dataStream, 8);
+        setupSink(dataStream);
+
+        // Run
+        env.execute();
+
+        // Check for data in Pinot
+        checkForDataInPinotWithRetry(rawData);
+    }
+
+    /**
+     * Tests the STREAMING execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testStreamingSink() throws Exception {
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.STREAMING);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+        env.setParallelism(2);
+        env.enableCheckpointing(STREAMING_CHECKPOINTING_INTERVAL);
+
+        List<String> rawData = getRawTestData(20);
+        DataStream<SingleColumnTableRow> dataStream = setupStreamingDataSource(env, rawData);
+        setupSink(dataStream);
+
+        // Start execution of job
+        env.executeAsync();
+
+        // Check for data in Pinot
+        checkForDataInPinotWithRetry(rawData);
+
+        // Generator source can now shut down
+        latch.countDown();
+    }
+
+    /**
+     * Tests failure recovery of the {@link PinotSink} using STREAMING execution mode.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testFailureRecoveryInStreamingSink() throws Exception {
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.STREAMING);
+        env.setParallelism(1);
+        env.enableCheckpointing(STREAMING_CHECKPOINTING_INTERVAL);
+
+        List<String> rawData = getRawTestData(20);
+        DataStream<SingleColumnTableRow> dataStream = setupStreamingDataSource(env, rawData);
+        // With a segment size of MAX_ROWS_PER_SEGMENT = 5 elements and a parallelism of 2,
+        // the failure will be raised once the first 2 segments were committed to Pinot.
+        dataStream = setupFailingMapper(dataStream, 12);
+        setupSink(dataStream);
+
+        // Start execution of job
+        env.executeAsync();
+
+        // Check for data in Pinot
+        checkForDataInPinotWithRetry(rawData);
+
+        // Generator source can now shut down
+        latch.countDown();
+    }
+
+    /**
+     * Generates a small test dataset consisting of {@link SingleColumnTableRow}s.
+     *
+     * @return List of SingleColumnTableRow
+     */
+    private List<String> getRawTestData(int numItems) {
+        return IntStream.range(1, numItems + 1)
+                .mapToObj(num -> "ColValue" + num)
+                .collect(Collectors.toList());
+    }
+
+    /**
+     * Setup the data source for STREAMING tests.
+     *
+     * @param env           Stream execution environment
+     * @param rawDataValues Data values to send
+     * @return resulting data stream
+     */
+    private DataStream<SingleColumnTableRow> setupStreamingDataSource(StreamExecutionEnvironment env, List<String> rawDataValues) {
+        SimpleStreamingSource source = new SimpleStreamingSource(rawDataValues, 10);
+        return env.addSource(source)
+                .name("Test input");
+    }
+
+    /**
+     * Setup the data source for BATCH tests.
+     *
+     * @param env           Stream execution environment
+     * @param rawDataValues Data values to send
+     * @return resulting data stream
+     */
+    private DataStream<SingleColumnTableRow> setupBatchDataSource(StreamExecutionEnvironment env, List<String> rawDataValues) {
+        return env.fromCollection(rawDataValues)
+                .map(value -> new SingleColumnTableRow(value, System.currentTimeMillis()))
+                .name("Test input");
+    }
+
+    /**
+     * Setup a mapper that fails when processing the nth element with n = failOnceAtNthElement.
+     *
+     * @param dataStream           Input data stream
+     * @param failOnceAtNthElement Number of elements to process before raising the exception
+     * @return resulting data stream
+     */
+    private DataStream<SingleColumnTableRow> setupFailingMapper(DataStream<SingleColumnTableRow> dataStream, int failOnceAtNthElement) {
+        AtomicInteger messageCounter = new AtomicInteger(0);
+
+        return dataStream.map(element -> {
+            if (!hasFailedOnce.get() && messageCounter.incrementAndGet() == failOnceAtNthElement) {
+                hasFailedOnce.set(true);
+                // Wait more than STREAMING_CHECKPOINTING_INTERVAL to ensure
+                // that at least one checkpoint was created before raising the exception.
+                Thread.sleep(4 * STREAMING_CHECKPOINTING_INTERVAL);

Review comment:
       I've removed the timeout by extending the source to support exception raising and added the following comment to explain why we wait for a snapshot creation. I hope this clears things up.
   > This allows to check whether the snapshot creation and failure recovery in
   > {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter} works properly,
   > respecting the already committed elements and those that are stored in an active
   > {@link org.apache.flink.streaming.connectors.pinot.writer.PinotWriterSegment}. Committed 
   > elements must not be saved to the snapshot while those in an active segment must be saved 
   > to the snapshot in order to enable later-on recovery.




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594300967



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.info("Posting string entity {} to {}", body, path);

Review comment:
       done

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.info("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {
+        HttpGet httpget = new HttpGet(this.controllerHostPort + path);
+        LOG.info("Sending GET request to {}", path);

Review comment:
       done




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594316546



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/segment/name/PinotSinkSegmentNameGenerator.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.flink.streaming.connectors.pinot.segment.name;
+
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+
+import javax.annotation.Nullable;
+import java.io.Serializable;
+
+/**
+ * Defines the segment name generator interface that is used to generate segment names.
+ */
+public abstract class PinotSinkSegmentNameGenerator implements SegmentNameGenerator, Serializable {

Review comment:
       We require the segment name generator to be serializable. This class just defines the required interface which users can inherit from to create their custom name generator.




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

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



[GitHub] [bahir-flink] fapaul commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
fapaul commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r593341789



##########
File path: flink-connector-pinot/README.md
##########
@@ -0,0 +1,60 @@
+# Flink Pinot Connector
+
+This connector provides a source and sink to [Apache Pinot](http://pinot.apache.org/)™.  

Review comment:
       ```suggestion
   This connector provides a sink to [Apache Pinot](http://pinot.apache.org/)™.  
   ```

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {

Review comment:
       I do not like the inheritance model here. WDYT about splitting this class into two classes? One holding the http client and exposing the `post`, `get` and `delete` method and the other class only holding production code like `tableHasSegment` etc. The basic http class is then also usable for testing and you can pass it to the `PinotTestHelper` to implement more API calls.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();

Review comment:
       WDYT of making the `PinotControllerApi` closeable? This way we can only instantiate one `CloseableHttpClient` in the constructor and do not have to create one for every request.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.info("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {
+        HttpGet httpget = new HttpGet(this.controllerHostPort + path);
+        LOG.info("Sending GET request to {}", path);
+        return this.execute(httpget);
+    }
+
+    /**
+     * Issues a DELETE request to the Pinot controller API.
+     *
+     * @param path Path to issue DELETE request to
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse delete(String path) throws IOException {
+        HttpDelete httpdelete = new HttpDelete(this.controllerHostPort + path);
+        LOG.info("Sending DELETE request to {}", path);
+        return this.execute(httpdelete);
+    }
+
+    /**
+     * Checks whether the provided segment name is registered with the given table.
+     *
+     * @param tableName   Target table's name
+     * @param segmentName Segment name to check
+     * @return True if segment with the provided name exists
+     * @throws IOException
+     */
+    public boolean tableHasSegment(String tableName, String segmentName) throws IOException {
+        ApiResponse res = this.get(String.format("/tables/%s/%s/metadata", tableName, segmentName));
+
+        if (res.statusLine.getStatusCode() == 200) {
+            // A segment named `segmentName` exists within the table named `tableName`
+            return true;
+        }
+        if (res.statusLine.getStatusCode() == 404) {
+            // There is no such segment named `segmentName` within the table named `tableName`
+            // (or the table named `tableName` does not exist)
+            return false;
+        }
+
+        // Received an unexpected status code
+        throw new PinotControllerApiException(res.responseBody);
+    }
+
+    /**
+     * Deletes a segment from a table.
+     *
+     * @param tableName   Target table's name
+     * @param segmentName Identifies the segment to delete
+     * @throws IOException
+     */
+    public void deleteSegment(String tableName, String segmentName) throws IOException {
+        ApiResponse res = this.delete(String.format("/tables/%s/%s", tableName, segmentName));
+
+        if (res.statusLine.getStatusCode() != 200) {
+            LOG.error("Could not delete segment {} from table {}. Pinot controller returned: {}", tableName, segmentName, res.responseBody);
+            throw new PinotControllerApiException(res.responseBody);
+        }
+    }
+
+    /**
+     * Fetches a Pinot table's schema via the Pinot controller API.
+     *
+     * @param tableName Target table's name
+     * @return Pinot table schema
+     * @throws IOException
+     */
+    public Schema getSchema(String tableName) throws IOException {
+        Schema schema;
+        ApiResponse res = this.get(String.format("/tables/%s/schema", tableName));
+        LOG.info("Get schema request for table {} returned {}", tableName, res.responseBody);
+
+        if (res.statusLine.getStatusCode() != 200) {
+            throw new PinotControllerApiException(res.responseBody);
+        }
+
+        try {
+            schema = JsonUtils.stringToObject(res.responseBody, Schema.class);
+        } catch (Exception e) {
+            throw new IllegalStateException("Caught exception while reading schema from Pinot Controller's response: " + res.responseBody, e);
+        }
+        LOG.info("Retrieved schema: {}", schema.toSingleLineJsonString());
+        return schema;
+    }
+
+    /**
+     * Fetches a Pinot table's configuration via the Pinot controller API.
+     *
+     * @param tableName Target table's name
+     * @return Pinot table configuration
+     * @throws IOException
+     */
+    public TableConfig getTableConfig(String tableName) throws IOException {
+        TableConfig tableConfig;
+        ApiResponse res = this.get(String.format("/tables/%s", tableName));
+        LOG.info("Get table config request for table {} returned {}", tableName, res.responseBody);
+
+        try {
+            String tableConfigAsJson = JsonUtils.stringToJsonNode(res.responseBody).get("OFFLINE").toString();
+            tableConfig = JsonUtils.stringToObject(tableConfigAsJson, TableConfig.class);
+        } catch (Exception e) {
+            throw new IllegalStateException("Caught exception while reading table config from Pinot Controller's response: " + res.responseBody, e);

Review comment:
       Why is this not a `PinotControllerApiException`?

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.info("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {
+        HttpGet httpget = new HttpGet(this.controllerHostPort + path);
+        LOG.info("Sending GET request to {}", path);
+        return this.execute(httpget);
+    }
+
+    /**
+     * Issues a DELETE request to the Pinot controller API.
+     *
+     * @param path Path to issue DELETE request to
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse delete(String path) throws IOException {
+        HttpDelete httpdelete = new HttpDelete(this.controllerHostPort + path);
+        LOG.info("Sending DELETE request to {}", path);
+        return this.execute(httpdelete);
+    }
+
+    /**
+     * Checks whether the provided segment name is registered with the given table.
+     *
+     * @param tableName   Target table's name
+     * @param segmentName Segment name to check
+     * @return True if segment with the provided name exists
+     * @throws IOException
+     */
+    public boolean tableHasSegment(String tableName, String segmentName) throws IOException {
+        ApiResponse res = this.get(String.format("/tables/%s/%s/metadata", tableName, segmentName));
+
+        if (res.statusLine.getStatusCode() == 200) {
+            // A segment named `segmentName` exists within the table named `tableName`
+            return true;
+        }
+        if (res.statusLine.getStatusCode() == 404) {
+            // There is no such segment named `segmentName` within the table named `tableName`
+            // (or the table named `tableName` does not exist)
+            return false;
+        }
+
+        // Received an unexpected status code
+        throw new PinotControllerApiException(res.responseBody);
+    }
+
+    /**
+     * Deletes a segment from a table.
+     *
+     * @param tableName   Target table's name
+     * @param segmentName Identifies the segment to delete
+     * @throws IOException
+     */
+    public void deleteSegment(String tableName, String segmentName) throws IOException {
+        ApiResponse res = this.delete(String.format("/tables/%s/%s", tableName, segmentName));
+
+        if (res.statusLine.getStatusCode() != 200) {
+            LOG.error("Could not delete segment {} from table {}. Pinot controller returned: {}", tableName, segmentName, res.responseBody);
+            throw new PinotControllerApiException(res.responseBody);
+        }
+    }
+
+    /**
+     * Fetches a Pinot table's schema via the Pinot controller API.
+     *
+     * @param tableName Target table's name
+     * @return Pinot table schema
+     * @throws IOException
+     */
+    public Schema getSchema(String tableName) throws IOException {
+        Schema schema;
+        ApiResponse res = this.get(String.format("/tables/%s/schema", tableName));
+        LOG.info("Get schema request for table {} returned {}", tableName, res.responseBody);
+
+        if (res.statusLine.getStatusCode() != 200) {
+            throw new PinotControllerApiException(res.responseBody);
+        }
+
+        try {
+            schema = JsonUtils.stringToObject(res.responseBody, Schema.class);
+        } catch (Exception e) {
+            throw new IllegalStateException("Caught exception while reading schema from Pinot Controller's response: " + res.responseBody, e);
+        }
+        LOG.info("Retrieved schema: {}", schema.toSingleLineJsonString());

Review comment:
       debug log

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.info("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {
+        HttpGet httpget = new HttpGet(this.controllerHostPort + path);
+        LOG.info("Sending GET request to {}", path);
+        return this.execute(httpget);
+    }
+
+    /**
+     * Issues a DELETE request to the Pinot controller API.
+     *
+     * @param path Path to issue DELETE request to
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse delete(String path) throws IOException {
+        HttpDelete httpdelete = new HttpDelete(this.controllerHostPort + path);
+        LOG.info("Sending DELETE request to {}", path);
+        return this.execute(httpdelete);
+    }
+
+    /**
+     * Checks whether the provided segment name is registered with the given table.
+     *
+     * @param tableName   Target table's name
+     * @param segmentName Segment name to check
+     * @return True if segment with the provided name exists
+     * @throws IOException
+     */
+    public boolean tableHasSegment(String tableName, String segmentName) throws IOException {
+        ApiResponse res = this.get(String.format("/tables/%s/%s/metadata", tableName, segmentName));
+
+        if (res.statusLine.getStatusCode() == 200) {
+            // A segment named `segmentName` exists within the table named `tableName`
+            return true;
+        }
+        if (res.statusLine.getStatusCode() == 404) {
+            // There is no such segment named `segmentName` within the table named `tableName`
+            // (or the table named `tableName` does not exist)
+            return false;
+        }
+
+        // Received an unexpected status code
+        throw new PinotControllerApiException(res.responseBody);
+    }
+
+    /**
+     * Deletes a segment from a table.
+     *
+     * @param tableName   Target table's name
+     * @param segmentName Identifies the segment to delete
+     * @throws IOException
+     */
+    public void deleteSegment(String tableName, String segmentName) throws IOException {
+        ApiResponse res = this.delete(String.format("/tables/%s/%s", tableName, segmentName));
+
+        if (res.statusLine.getStatusCode() != 200) {
+            LOG.error("Could not delete segment {} from table {}. Pinot controller returned: {}", tableName, segmentName, res.responseBody);
+            throw new PinotControllerApiException(res.responseBody);
+        }
+    }
+
+    /**
+     * Fetches a Pinot table's schema via the Pinot controller API.
+     *
+     * @param tableName Target table's name
+     * @return Pinot table schema
+     * @throws IOException
+     */
+    public Schema getSchema(String tableName) throws IOException {
+        Schema schema;
+        ApiResponse res = this.get(String.format("/tables/%s/schema", tableName));
+        LOG.info("Get schema request for table {} returned {}", tableName, res.responseBody);
+
+        if (res.statusLine.getStatusCode() != 200) {
+            throw new PinotControllerApiException(res.responseBody);
+        }
+
+        try {
+            schema = JsonUtils.stringToObject(res.responseBody, Schema.class);
+        } catch (Exception e) {
+            throw new IllegalStateException("Caught exception while reading schema from Pinot Controller's response: " + res.responseBody, e);
+        }
+        LOG.info("Retrieved schema: {}", schema.toSingleLineJsonString());
+        return schema;
+    }
+
+    /**
+     * Fetches a Pinot table's configuration via the Pinot controller API.
+     *
+     * @param tableName Target table's name
+     * @return Pinot table configuration
+     * @throws IOException
+     */
+    public TableConfig getTableConfig(String tableName) throws IOException {
+        TableConfig tableConfig;
+        ApiResponse res = this.get(String.format("/tables/%s", tableName));
+        LOG.info("Get table config request for table {} returned {}", tableName, res.responseBody);

Review comment:
       debug log

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);

Review comment:
       How is it possible to get here? I would assume every recovery deletes segments which have been committed previously.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());

Review comment:
       debug log

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);

Review comment:
       Can the schema change over the lifetime of a table? If it is immutable we can retrieve the schema once during init.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);

Review comment:
       IMO once one of the SegmentCommitter fails we should also fail the pipeline. Otherwise we can accumulate globalcommittables endlessly. WDYT of returning an optional exception from the SegmentCommitter and rethrow it?

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/segment/name/PinotSinkSegmentNameGenerator.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.flink.streaming.connectors.pinot.segment.name;
+
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+
+import javax.annotation.Nullable;
+import java.io.Serializable;
+
+/**
+ * Defines the segment name generator interface that is used to generate segment names.
+ */
+public abstract class PinotSinkSegmentNameGenerator implements SegmentNameGenerator, Serializable {

Review comment:
       Why did you make this abstract class as far as I see it does not offer more than the `SegmentNameGenerator` interface? Probably also worthwhile mentioning in the java doc.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        public CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    static class SegmentCommitter implements Callable<Boolean> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class);
+
+        final String pinotControllerHost;
+        final String pinotControllerPort;
+        final String tempDirPrefix;
+        final FileSystemAdapter fsAdapter;
+        final String dataFilePath;
+        final String segmentName;
+        final Schema tableSchema;
+        final TableConfig tableConfig;
+        final String timeColumnName;
+        final TimeUnit segmentTimeUnit;
+
+        /**
+         * @param pinotControllerHost Host of the Pinot controller
+         * @param pinotControllerPort Port of the Pinot controller
+         * @param fsAdapter           Filesystem adapter used to load data files from the shared file system
+         * @param dataFilePath        Data file to load from the shared file system
+         * @param segmentName         Name of the segment to create and commit
+         * @param tableSchema         Pinot table schema
+         * @param tableConfig         Pinot table config
+         * @param timeColumnName      Name of the column containing the timestamp
+         * @param segmentTimeUnit     Unit of the time column
+         */
+        public SegmentCommitter(String pinotControllerHost, String pinotControllerPort, String tempDirPrefix, FileSystemAdapter fsAdapter, String dataFilePath, String segmentName, Schema tableSchema, TableConfig tableConfig, String timeColumnName, TimeUnit segmentTimeUnit) {
+            this.pinotControllerHost = pinotControllerHost;
+            this.pinotControllerPort = pinotControllerPort;
+            this.tempDirPrefix = tempDirPrefix;
+            this.fsAdapter = fsAdapter;
+            this.dataFilePath = dataFilePath;
+            this.segmentName = segmentName;
+            this.tableSchema = tableSchema;
+            this.tableConfig = tableConfig;
+            this.timeColumnName = timeColumnName;
+            this.segmentTimeUnit = segmentTimeUnit;
+        }
+
+        /**
+         * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment
+         * and finally uploads the segment to the Pinot controller
+         *
+         * @return True if the commit succeeded
+         */
+        @Override
+        public Boolean call() {
+            try {
+                // Download data file from the shared filesystem
+                LOG.info("Downloading data file {} from shared file system...", dataFilePath);
+                File segmentData = fsAdapter.copyToLocalFile(dataFilePath);
+                LOG.info("Successfully downloaded data file {} from shared file system", dataFilePath);
+
+                File segmentFile = Files.createTempDirectory(this.tempDirPrefix).toFile();

Review comment:
       Who is responsible to cleanup the temp directory? Can we create one tmp directory for all SegmentCommitters and delete the directory when finished?

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.info("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {
+        HttpGet httpget = new HttpGet(this.controllerHostPort + path);
+        LOG.info("Sending GET request to {}", path);

Review comment:
       Should be debug log.

##########
File path: flink-connector-pinot/README.md
##########
@@ -0,0 +1,60 @@
+# Flink Pinot Connector
+
+This connector provides a source and sink to [Apache Pinot](http://pinot.apache.org/)™.  
+To use this connector, add the following dependency to your project:
+
+    <dependency>
+      <groupId>org.apache.bahir</groupId>
+      <artifactId>flink-connector-pinot_2.11</artifactId>
+      <version>1.1-SNAPSHOT</version>
+    </dependency>
+
+*Version Compatibility*: This module is compatible with Pinot 0.6.0.
+
+Note that the streaming connectors are not part of the binary distribution of Flink. You need to link them into your job jar for cluster execution.

Review comment:
       I am missing in the readme a few of the design decisions.
   
   - How does streaming/batch mode work?
   - At least once delivery?
   - Core architecture overview (Commiter, GlobalCommitter)
   
   You can look at https://github.com/apache/bahir-flink/blob/521df6c23f0a2a788d0853a260d31caaca33f31b/flink-connector-influxdb2/README.md to get some ideas although I do think not all of that is necessary ;)

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.info("Posting string entity {} to {}", body, path);

Review comment:
       Should be debug log.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.info("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {
+        HttpGet httpget = new HttpGet(this.controllerHostPort + path);
+        LOG.info("Sending GET request to {}", path);
+        return this.execute(httpget);
+    }
+
+    /**
+     * Issues a DELETE request to the Pinot controller API.
+     *
+     * @param path Path to issue DELETE request to
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse delete(String path) throws IOException {
+        HttpDelete httpdelete = new HttpDelete(this.controllerHostPort + path);
+        LOG.info("Sending DELETE request to {}", path);

Review comment:
       debug log

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import com.google.common.collect.Iterables;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Accepts incoming elements and creates {@link PinotSinkCommittable}s out of them on request.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotSinkWriter<IN> implements SinkWriter<IN, PinotSinkCommittable, Void> {
+
+    private static final Logger LOG = LoggerFactory.getLogger("PinotSinkWriter");
+
+    private final int maxRowsPerSegment;
+    private EventTimeExtractor<IN> eventTimeExtractor;
+    private final String tempDirPrefix;
+    private final JsonSerializer<IN> jsonSerializer;
+
+    private final List<PinotWriterSegment<IN>> activeSegments;
+    private final FileSystemAdapter fsAdapter;
+
+    private final int subtaskId;
+
+    /**
+     * @param subtaskId          Subtask id provided by Flink
+     * @param maxRowsPerSegment  Maximum number of rows to be stored within a Pinot segment
+     * @param eventTimeExtractor Defines the way event times are extracted from received objects
+     * @param tempDirPrefix      Prefix for temp directories used
+     * @param jsonSerializer     Serializer used to convert elements to JSON
+     * @param fsAdapter          Filesystem adapter used to save files for sharing files across nodes
+     */
+    public PinotSinkWriter(int subtaskId, int maxRowsPerSegment, EventTimeExtractor<IN> eventTimeExtractor, String tempDirPrefix, JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        this.subtaskId = subtaskId;
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.eventTimeExtractor = checkNotNull(eventTimeExtractor);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.activeSegments = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements from an upstream tasks and writes them into {@link PinotWriterSegment}
+     *
+     * @param element Object from upstream task
+     * @param context SinkWriter context
+     * @throws IOException
+     */
+    @Override
+    public void write(IN element, Context context) throws IOException {
+        final PinotWriterSegment<IN> inProgressSegment = this.getOrCreateInProgressSegment();
+        inProgressSegment.write(element, this.eventTimeExtractor.getEventTime(element, context));
+    }
+
+    /**
+     * Creates {@link PinotSinkCommittable}s from elements received via {@link #write}
+     *
+     * @param flush Flush all currently known elements into the {@link PinotSinkCommittable}s
+     * @return List of {@link PinotSinkCommittable} to process in {@link org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter}
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkCommittable> prepareCommit(boolean flush) throws IOException {
+        List<PinotWriterSegment<IN>> segmentsToCommit = this.activeSegments.stream()
+                .filter(s -> flush || !s.acceptsElements())
+                .collect(Collectors.toList());
+        LOG.info("Identified {} segments to commit [subtaskId={}]", segmentsToCommit.size(), this.subtaskId);
+
+        LOG.info("Creating committables... [subtaskId={}]", subtaskId);
+        List<PinotSinkCommittable> committables = new ArrayList<>();
+        for (final PinotWriterSegment<IN> segment : segmentsToCommit) {
+            committables.add(segment.prepareCommit());
+        }
+        LOG.info("Created {} committables [subtaskId={}]", committables.size(), subtaskId);
+
+        this.activeSegments.removeAll(segmentsToCommit);
+        return committables;
+    }
+
+    /**
+     * Gets the {@link PinotWriterSegment} still accepting elements or creates a new one.
+     *
+     * @return {@link PinotWriterSegment} accepting at least one more element
+     */
+    private PinotWriterSegment<IN> getOrCreateInProgressSegment() {
+        final PinotWriterSegment<IN> latestSegment = Iterables.getLast(this.activeSegments, null);
+        if (latestSegment == null || !latestSegment.acceptsElements()) {
+            final PinotWriterSegment<IN> inProgressSegment = new PinotWriterSegment<>(this.maxRowsPerSegment, this.tempDirPrefix, this.jsonSerializer, this.fsAdapter);
+            this.activeSegments.add(inProgressSegment);
+            return inProgressSegment;
+        }
+        return latestSegment;
+    }
+
+    /**
+     * Snapshots the current state to be stored within a checkpoint. As we do not need to save any
+     * information in snapshots, this method always returns an empty ArrayList.
+     *
+     * @return always an empty ArrayList
+     */
+    @Override
+    public List<Void> snapshotState() {
+        return new ArrayList<>();
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {

Review comment:
       Nit: clean up the temporary written files?

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/EmulatedPinotSinkTest.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.filesystem.LocalFileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.pinot.client.ResultSet;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * E2e tests for Pinot Sink using BATCH and STREAMING execution mode
+ */
+public class EmulatedPinotSinkTest extends PinotTestBase {

Review comment:
       Overall the timeouts make the code unreliable and make is runtime somewhat unpredictable and flaky on slower systems. If possible you should replace all with better synchronization mechanics

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);

Review comment:
       We should instantiate the controller API in the constructor.

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.util.TestLogger;
+import org.apache.pinot.spi.config.table.*;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.Wait;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Base class for PinotSink e2e tests
+ */
+@Testcontainers
+public class PinotTestBase extends TestLogger implements Serializable {

Review comment:
       Why Serializable?

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);

Review comment:
       We should instantiate the controller API in the constructor.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.info("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {
+        HttpGet httpget = new HttpGet(this.controllerHostPort + path);
+        LOG.info("Sending GET request to {}", path);
+        return this.execute(httpget);
+    }
+
+    /**
+     * Issues a DELETE request to the Pinot controller API.
+     *
+     * @param path Path to issue DELETE request to
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse delete(String path) throws IOException {
+        HttpDelete httpdelete = new HttpDelete(this.controllerHostPort + path);
+        LOG.info("Sending DELETE request to {}", path);
+        return this.execute(httpdelete);
+    }
+
+    /**
+     * Checks whether the provided segment name is registered with the given table.
+     *
+     * @param tableName   Target table's name
+     * @param segmentName Segment name to check
+     * @return True if segment with the provided name exists
+     * @throws IOException
+     */
+    public boolean tableHasSegment(String tableName, String segmentName) throws IOException {
+        ApiResponse res = this.get(String.format("/tables/%s/%s/metadata", tableName, segmentName));
+
+        if (res.statusLine.getStatusCode() == 200) {
+            // A segment named `segmentName` exists within the table named `tableName`
+            return true;
+        }
+        if (res.statusLine.getStatusCode() == 404) {
+            // There is no such segment named `segmentName` within the table named `tableName`
+            // (or the table named `tableName` does not exist)
+            return false;
+        }
+
+        // Received an unexpected status code
+        throw new PinotControllerApiException(res.responseBody);
+    }
+
+    /**
+     * Deletes a segment from a table.
+     *
+     * @param tableName   Target table's name
+     * @param segmentName Identifies the segment to delete
+     * @throws IOException
+     */
+    public void deleteSegment(String tableName, String segmentName) throws IOException {
+        ApiResponse res = this.delete(String.format("/tables/%s/%s", tableName, segmentName));
+
+        if (res.statusLine.getStatusCode() != 200) {
+            LOG.error("Could not delete segment {} from table {}. Pinot controller returned: {}", tableName, segmentName, res.responseBody);
+            throw new PinotControllerApiException(res.responseBody);
+        }
+    }
+
+    /**
+     * Fetches a Pinot table's schema via the Pinot controller API.
+     *
+     * @param tableName Target table's name
+     * @return Pinot table schema
+     * @throws IOException
+     */
+    public Schema getSchema(String tableName) throws IOException {
+        Schema schema;
+        ApiResponse res = this.get(String.format("/tables/%s/schema", tableName));
+        LOG.info("Get schema request for table {} returned {}", tableName, res.responseBody);
+
+        if (res.statusLine.getStatusCode() != 200) {
+            throw new PinotControllerApiException(res.responseBody);
+        }
+
+        try {
+            schema = JsonUtils.stringToObject(res.responseBody, Schema.class);
+        } catch (Exception e) {
+            throw new IllegalStateException("Caught exception while reading schema from Pinot Controller's response: " + res.responseBody, e);
+        }
+        LOG.info("Retrieved schema: {}", schema.toSingleLineJsonString());
+        return schema;
+    }
+
+    /**
+     * Fetches a Pinot table's configuration via the Pinot controller API.
+     *
+     * @param tableName Target table's name
+     * @return Pinot table configuration
+     * @throws IOException
+     */
+    public TableConfig getTableConfig(String tableName) throws IOException {
+        TableConfig tableConfig;
+        ApiResponse res = this.get(String.format("/tables/%s", tableName));
+        LOG.info("Get table config request for table {} returned {}", tableName, res.responseBody);
+
+        try {
+            String tableConfigAsJson = JsonUtils.stringToJsonNode(res.responseBody).get("OFFLINE").toString();
+            tableConfig = JsonUtils.stringToObject(tableConfigAsJson, TableConfig.class);
+        } catch (Exception e) {
+            throw new IllegalStateException("Caught exception while reading table config from Pinot Controller's response: " + res.responseBody, e);
+        }
+        LOG.info("Retrieved table config: {}", tableConfig.toJsonString());

Review comment:
       debug log

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);

Review comment:
       You should instantiate the controller API in the constructor.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);

Review comment:
       We should instantiate the controller API in the constructor.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {

Review comment:
       What kind of Exceptions are you trying hide here? :) In general it is discouraged to use a **gonna catch'em all** block.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {

Review comment:
       WDYT of changing this loop to an index based one? You can retrieve the `dataFilePath` as follows
   
   `String dataFilePath = globalCommittable.getDataFilePaths().get(sequenceId)`
   
   if sequenceId is the index variable.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());

Review comment:
       Provide a detailed error message and have the exception as second parameter.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();

Review comment:
       You need to handle the lifecycle of the thread pool properly.
   
   1. Only instantiate it once in the constructor
   2. Limit the thread pool size IMO it needs to be configurable and have a sane default value
   3. Handle shutdown of the threadpool. When the GlobalCommitter is closed you can call `shutdown()`
   
   Overall I am not super convinced by having a threading model in the sink. I would rather find a way to leverage Flink's dataflow model to parallelize committing. 

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {

Review comment:
       private?

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        public CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    static class SegmentCommitter implements Callable<Boolean> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class);
+
+        final String pinotControllerHost;
+        final String pinotControllerPort;
+        final String tempDirPrefix;
+        final FileSystemAdapter fsAdapter;
+        final String dataFilePath;
+        final String segmentName;
+        final Schema tableSchema;
+        final TableConfig tableConfig;
+        final String timeColumnName;
+        final TimeUnit segmentTimeUnit;
+
+        /**
+         * @param pinotControllerHost Host of the Pinot controller
+         * @param pinotControllerPort Port of the Pinot controller
+         * @param fsAdapter           Filesystem adapter used to load data files from the shared file system
+         * @param dataFilePath        Data file to load from the shared file system
+         * @param segmentName         Name of the segment to create and commit
+         * @param tableSchema         Pinot table schema
+         * @param tableConfig         Pinot table config
+         * @param timeColumnName      Name of the column containing the timestamp
+         * @param segmentTimeUnit     Unit of the time column
+         */
+        public SegmentCommitter(String pinotControllerHost, String pinotControllerPort, String tempDirPrefix, FileSystemAdapter fsAdapter, String dataFilePath, String segmentName, Schema tableSchema, TableConfig tableConfig, String timeColumnName, TimeUnit segmentTimeUnit) {
+            this.pinotControllerHost = pinotControllerHost;
+            this.pinotControllerPort = pinotControllerPort;
+            this.tempDirPrefix = tempDirPrefix;
+            this.fsAdapter = fsAdapter;
+            this.dataFilePath = dataFilePath;
+            this.segmentName = segmentName;
+            this.tableSchema = tableSchema;
+            this.tableConfig = tableConfig;
+            this.timeColumnName = timeColumnName;
+            this.segmentTimeUnit = segmentTimeUnit;
+        }
+
+        /**
+         * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment
+         * and finally uploads the segment to the Pinot controller
+         *
+         * @return True if the commit succeeded
+         */
+        @Override
+        public Boolean call() {
+            try {
+                // Download data file from the shared filesystem
+                LOG.info("Downloading data file {} from shared file system...", dataFilePath);

Review comment:
       A lot of debug logs

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        public CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    static class SegmentCommitter implements Callable<Boolean> {

Review comment:
       private?

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/external/JsonSerializer.java
##########
@@ -0,0 +1,8 @@
+package org.apache.flink.streaming.connectors.pinot.external;
+
+import java.io.Serializable;
+
+public abstract class JsonSerializer<IN> implements Serializable {

Review comment:
       A java doc would help to understand why we have so serialize everything to json.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkCommittableSerializer.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.flink.streaming.connectors.pinot.serializer;
+
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+
+/**
+ * Serializer for {@link PinotSinkCommittable}
+ */
+public class PinotSinkCommittableSerializer implements SimpleVersionedSerializer<PinotSinkCommittable> {
+
+    private static final int CURRENT_VERSION = 0;
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(PinotSinkCommittable pinotSinkCommittable) {
+        return SerializationUtils.serialize(pinotSinkCommittable);
+    }
+
+    @Override
+    public PinotSinkCommittable deserialize(int i, byte[] bytes) {

Review comment:
       Please add a simple check as in https://github.com/apache/flink/blob/ab2f89940ddbf71cbc074f2ce758696f4f53cd62/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/sink/FileSinkCommittableSerializer.java#L70 to assure we serialize the correct version.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/external/JsonSerializer.java
##########
@@ -0,0 +1,8 @@
+package org.apache.flink.streaming.connectors.pinot.external;

Review comment:
       Missing license.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        public CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    static class SegmentCommitter implements Callable<Boolean> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class);
+
+        final String pinotControllerHost;
+        final String pinotControllerPort;
+        final String tempDirPrefix;
+        final FileSystemAdapter fsAdapter;
+        final String dataFilePath;
+        final String segmentName;
+        final Schema tableSchema;
+        final TableConfig tableConfig;
+        final String timeColumnName;
+        final TimeUnit segmentTimeUnit;
+
+        /**
+         * @param pinotControllerHost Host of the Pinot controller
+         * @param pinotControllerPort Port of the Pinot controller
+         * @param fsAdapter           Filesystem adapter used to load data files from the shared file system
+         * @param dataFilePath        Data file to load from the shared file system
+         * @param segmentName         Name of the segment to create and commit
+         * @param tableSchema         Pinot table schema
+         * @param tableConfig         Pinot table config
+         * @param timeColumnName      Name of the column containing the timestamp
+         * @param segmentTimeUnit     Unit of the time column
+         */
+        public SegmentCommitter(String pinotControllerHost, String pinotControllerPort, String tempDirPrefix, FileSystemAdapter fsAdapter, String dataFilePath, String segmentName, Schema tableSchema, TableConfig tableConfig, String timeColumnName, TimeUnit segmentTimeUnit) {
+            this.pinotControllerHost = pinotControllerHost;
+            this.pinotControllerPort = pinotControllerPort;
+            this.tempDirPrefix = tempDirPrefix;
+            this.fsAdapter = fsAdapter;
+            this.dataFilePath = dataFilePath;
+            this.segmentName = segmentName;
+            this.tableSchema = tableSchema;
+            this.tableConfig = tableConfig;
+            this.timeColumnName = timeColumnName;
+            this.segmentTimeUnit = segmentTimeUnit;
+        }
+
+        /**
+         * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment
+         * and finally uploads the segment to the Pinot controller
+         *
+         * @return True if the commit succeeded
+         */
+        @Override
+        public Boolean call() {
+            try {
+                // Download data file from the shared filesystem
+                LOG.info("Downloading data file {} from shared file system...", dataFilePath);
+                File segmentData = fsAdapter.copyToLocalFile(dataFilePath);
+                LOG.info("Successfully downloaded data file {} from shared file system", dataFilePath);
+
+                File segmentFile = Files.createTempDirectory(this.tempDirPrefix).toFile();
+                LOG.info("Creating segment in " + segmentFile.getAbsolutePath());
+
+                // Creates a segment with name `segmentName` in `segmentFile`
+                this.generateSegment(segmentData, segmentFile, true);
+
+                // Uploads the recently created segment to the Pinot controller
+                this.uploadSegment(segmentFile);
+
+                // Commit successful
+                return true;
+            } catch (IOException e) {
+                LOG.error(e.getMessage());
+
+                // Commit failed
+                return false;
+            }
+        }
+
+        /**
+         * Creates a segment from the given parameters.
+         * This method was adapted from {@link org.apache.pinot.tools.admin.command.CreateSegmentCommand}.
+         *
+         * @param dataFile                  File containing the JSON data
+         * @param outDir                    Segment target path
+         * @param _postCreationVerification Verify segment after generation
+         */
+        public void generateSegment(File dataFile, File outDir, Boolean _postCreationVerification) {
+            SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(tableConfig, tableSchema);
+            segmentGeneratorConfig.setSegmentName(segmentName);
+            segmentGeneratorConfig.setSegmentTimeUnit(segmentTimeUnit);
+            segmentGeneratorConfig.setTimeColumnName(timeColumnName);
+            segmentGeneratorConfig.setInputFilePath(dataFile.getPath());
+            segmentGeneratorConfig.setFormat(FileFormat.JSON);
+            segmentGeneratorConfig.setOutDir(outDir.getPath());
+            segmentGeneratorConfig.setTableName(tableConfig.getTableName());
+
+            try {
+                SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+                driver.init(segmentGeneratorConfig);
+                driver.build();
+                File indexDir = new File(outDir, segmentName);
+                LOG.info("Successfully created segment: {} in directory: {}", segmentName, indexDir);
+                if (_postCreationVerification) {
+                    LOG.info("Verifying the segment by loading it");
+                    ImmutableSegment segment = ImmutableSegmentLoader.load(indexDir, ReadMode.mmap);
+                    LOG.info("Successfully loaded segment: {} of size: {} bytes", segmentName,
+                            segment.getSegmentSizeBytes());
+                    segment.destroy();
+                }
+            } catch (Exception e) {

Review comment:
       Please only catch the needed exceptions.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkCommittableSerializer.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.flink.streaming.connectors.pinot.serializer;
+
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+
+/**
+ * Serializer for {@link PinotSinkCommittable}
+ */
+public class PinotSinkCommittableSerializer implements SimpleVersionedSerializer<PinotSinkCommittable> {
+
+    private static final int CURRENT_VERSION = 0;

Review comment:
       I think the versions always start with 1.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        public CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    static class SegmentCommitter implements Callable<Boolean> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class);
+
+        final String pinotControllerHost;
+        final String pinotControllerPort;
+        final String tempDirPrefix;
+        final FileSystemAdapter fsAdapter;
+        final String dataFilePath;
+        final String segmentName;
+        final Schema tableSchema;
+        final TableConfig tableConfig;
+        final String timeColumnName;
+        final TimeUnit segmentTimeUnit;
+
+        /**
+         * @param pinotControllerHost Host of the Pinot controller
+         * @param pinotControllerPort Port of the Pinot controller
+         * @param fsAdapter           Filesystem adapter used to load data files from the shared file system
+         * @param dataFilePath        Data file to load from the shared file system
+         * @param segmentName         Name of the segment to create and commit
+         * @param tableSchema         Pinot table schema
+         * @param tableConfig         Pinot table config
+         * @param timeColumnName      Name of the column containing the timestamp
+         * @param segmentTimeUnit     Unit of the time column
+         */
+        public SegmentCommitter(String pinotControllerHost, String pinotControllerPort, String tempDirPrefix, FileSystemAdapter fsAdapter, String dataFilePath, String segmentName, Schema tableSchema, TableConfig tableConfig, String timeColumnName, TimeUnit segmentTimeUnit) {
+            this.pinotControllerHost = pinotControllerHost;
+            this.pinotControllerPort = pinotControllerPort;
+            this.tempDirPrefix = tempDirPrefix;
+            this.fsAdapter = fsAdapter;
+            this.dataFilePath = dataFilePath;
+            this.segmentName = segmentName;
+            this.tableSchema = tableSchema;
+            this.tableConfig = tableConfig;
+            this.timeColumnName = timeColumnName;
+            this.segmentTimeUnit = segmentTimeUnit;
+        }
+
+        /**
+         * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment
+         * and finally uploads the segment to the Pinot controller
+         *
+         * @return True if the commit succeeded
+         */
+        @Override
+        public Boolean call() {
+            try {
+                // Download data file from the shared filesystem
+                LOG.info("Downloading data file {} from shared file system...", dataFilePath);
+                File segmentData = fsAdapter.copyToLocalFile(dataFilePath);
+                LOG.info("Successfully downloaded data file {} from shared file system", dataFilePath);
+
+                File segmentFile = Files.createTempDirectory(this.tempDirPrefix).toFile();
+                LOG.info("Creating segment in " + segmentFile.getAbsolutePath());
+
+                // Creates a segment with name `segmentName` in `segmentFile`
+                this.generateSegment(segmentData, segmentFile, true);
+
+                // Uploads the recently created segment to the Pinot controller
+                this.uploadSegment(segmentFile);
+
+                // Commit successful
+                return true;
+            } catch (IOException e) {
+                LOG.error(e.getMessage());
+
+                // Commit failed
+                return false;
+            }
+        }
+
+        /**
+         * Creates a segment from the given parameters.
+         * This method was adapted from {@link org.apache.pinot.tools.admin.command.CreateSegmentCommand}.
+         *
+         * @param dataFile                  File containing the JSON data
+         * @param outDir                    Segment target path
+         * @param _postCreationVerification Verify segment after generation
+         */
+        public void generateSegment(File dataFile, File outDir, Boolean _postCreationVerification) {
+            SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(tableConfig, tableSchema);
+            segmentGeneratorConfig.setSegmentName(segmentName);
+            segmentGeneratorConfig.setSegmentTimeUnit(segmentTimeUnit);
+            segmentGeneratorConfig.setTimeColumnName(timeColumnName);
+            segmentGeneratorConfig.setInputFilePath(dataFile.getPath());
+            segmentGeneratorConfig.setFormat(FileFormat.JSON);
+            segmentGeneratorConfig.setOutDir(outDir.getPath());
+            segmentGeneratorConfig.setTableName(tableConfig.getTableName());
+
+            try {
+                SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+                driver.init(segmentGeneratorConfig);
+                driver.build();
+                File indexDir = new File(outDir, segmentName);
+                LOG.info("Successfully created segment: {} in directory: {}", segmentName, indexDir);

Review comment:
       More debug logs

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkGlobalCommittableSerializer.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.flink.streaming.connectors.pinot.serializer;
+
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommittable;
+
+/**
+ * Serializer for {@link PinotSinkGlobalCommittable}
+ */
+public class PinotSinkGlobalCommittableSerializer implements SimpleVersionedSerializer<PinotSinkGlobalCommittable> {

Review comment:
       I guess serializing the complete committable is pretty cost intensive when doing it with the apache commons util.
   Nit: Other Flink sinks manually write the needed information which should also be possible in your case. https://github.com/apache/flink/blob/ab2f89940ddbf71cbc074f2ce758696f4f53cd62/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializer.java#L46

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/segment/name/SimpleSegmentNameGenerator.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.flink.streaming.connectors.pinot.segment.name;
+
+import javax.annotation.Nullable;
+
+/**
+ * Adapted from {@link org.apache.pinot.core.segment.name.SimpleSegmentNameGenerator}.
+ * <p>
+ * Simple segment name generator which does not perform time conversion.
+ * <p>
+ * The segment name is simply joining the following fields with '_' but ignoring all the {@code null}s.
+ * <ul>
+ *   <li>Table name</li>
+ *   <li>Minimum time value</li>
+ *   <li>Maximum time value</li>
+ *   <li>Segment name postfix</li>
+ *   <li>Sequence id</li>
+ * </ul>
+ */
+public class SimpleSegmentNameGenerator extends PinotSinkSegmentNameGenerator {
+
+    private final String tableName;
+    private final String segmentNamePostfix;
+
+    public SimpleSegmentNameGenerator(String tableName, String segmentNamePostfix) {
+        this.tableName = tableName;
+        this.segmentNamePostfix = segmentNamePostfix;

Review comment:
       checkNotNull

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        public CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    static class SegmentCommitter implements Callable<Boolean> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class);
+
+        final String pinotControllerHost;
+        final String pinotControllerPort;
+        final String tempDirPrefix;
+        final FileSystemAdapter fsAdapter;
+        final String dataFilePath;
+        final String segmentName;
+        final Schema tableSchema;
+        final TableConfig tableConfig;
+        final String timeColumnName;
+        final TimeUnit segmentTimeUnit;
+
+        /**
+         * @param pinotControllerHost Host of the Pinot controller
+         * @param pinotControllerPort Port of the Pinot controller
+         * @param fsAdapter           Filesystem adapter used to load data files from the shared file system
+         * @param dataFilePath        Data file to load from the shared file system
+         * @param segmentName         Name of the segment to create and commit
+         * @param tableSchema         Pinot table schema
+         * @param tableConfig         Pinot table config
+         * @param timeColumnName      Name of the column containing the timestamp
+         * @param segmentTimeUnit     Unit of the time column
+         */
+        public SegmentCommitter(String pinotControllerHost, String pinotControllerPort, String tempDirPrefix, FileSystemAdapter fsAdapter, String dataFilePath, String segmentName, Schema tableSchema, TableConfig tableConfig, String timeColumnName, TimeUnit segmentTimeUnit) {
+            this.pinotControllerHost = pinotControllerHost;
+            this.pinotControllerPort = pinotControllerPort;
+            this.tempDirPrefix = tempDirPrefix;
+            this.fsAdapter = fsAdapter;
+            this.dataFilePath = dataFilePath;
+            this.segmentName = segmentName;
+            this.tableSchema = tableSchema;
+            this.tableConfig = tableConfig;
+            this.timeColumnName = timeColumnName;
+            this.segmentTimeUnit = segmentTimeUnit;
+        }
+
+        /**
+         * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment
+         * and finally uploads the segment to the Pinot controller
+         *
+         * @return True if the commit succeeded
+         */
+        @Override
+        public Boolean call() {
+            try {
+                // Download data file from the shared filesystem
+                LOG.info("Downloading data file {} from shared file system...", dataFilePath);
+                File segmentData = fsAdapter.copyToLocalFile(dataFilePath);
+                LOG.info("Successfully downloaded data file {} from shared file system", dataFilePath);
+
+                File segmentFile = Files.createTempDirectory(this.tempDirPrefix).toFile();
+                LOG.info("Creating segment in " + segmentFile.getAbsolutePath());
+
+                // Creates a segment with name `segmentName` in `segmentFile`
+                this.generateSegment(segmentData, segmentFile, true);
+
+                // Uploads the recently created segment to the Pinot controller
+                this.uploadSegment(segmentFile);
+
+                // Commit successful
+                return true;
+            } catch (IOException e) {
+                LOG.error(e.getMessage());
+
+                // Commit failed
+                return false;
+            }
+        }
+
+        /**
+         * Creates a segment from the given parameters.
+         * This method was adapted from {@link org.apache.pinot.tools.admin.command.CreateSegmentCommand}.
+         *
+         * @param dataFile                  File containing the JSON data
+         * @param outDir                    Segment target path
+         * @param _postCreationVerification Verify segment after generation
+         */
+        public void generateSegment(File dataFile, File outDir, Boolean _postCreationVerification) {

Review comment:
       private?

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import com.google.common.collect.Iterables;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Accepts incoming elements and creates {@link PinotSinkCommittable}s out of them on request.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotSinkWriter<IN> implements SinkWriter<IN, PinotSinkCommittable, Void> {
+
+    private static final Logger LOG = LoggerFactory.getLogger("PinotSinkWriter");
+
+    private final int maxRowsPerSegment;
+    private EventTimeExtractor<IN> eventTimeExtractor;

Review comment:
       final?

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import com.google.common.collect.Iterables;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Accepts incoming elements and creates {@link PinotSinkCommittable}s out of them on request.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotSinkWriter<IN> implements SinkWriter<IN, PinotSinkCommittable, Void> {
+
+    private static final Logger LOG = LoggerFactory.getLogger("PinotSinkWriter");
+
+    private final int maxRowsPerSegment;
+    private EventTimeExtractor<IN> eventTimeExtractor;
+    private final String tempDirPrefix;
+    private final JsonSerializer<IN> jsonSerializer;
+
+    private final List<PinotWriterSegment<IN>> activeSegments;
+    private final FileSystemAdapter fsAdapter;
+
+    private final int subtaskId;
+
+    /**
+     * @param subtaskId          Subtask id provided by Flink
+     * @param maxRowsPerSegment  Maximum number of rows to be stored within a Pinot segment
+     * @param eventTimeExtractor Defines the way event times are extracted from received objects
+     * @param tempDirPrefix      Prefix for temp directories used
+     * @param jsonSerializer     Serializer used to convert elements to JSON
+     * @param fsAdapter          Filesystem adapter used to save files for sharing files across nodes
+     */
+    public PinotSinkWriter(int subtaskId, int maxRowsPerSegment, EventTimeExtractor<IN> eventTimeExtractor, String tempDirPrefix, JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        this.subtaskId = subtaskId;
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.eventTimeExtractor = checkNotNull(eventTimeExtractor);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.activeSegments = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements from an upstream tasks and writes them into {@link PinotWriterSegment}
+     *
+     * @param element Object from upstream task
+     * @param context SinkWriter context
+     * @throws IOException
+     */
+    @Override
+    public void write(IN element, Context context) throws IOException {
+        final PinotWriterSegment<IN> inProgressSegment = this.getOrCreateInProgressSegment();
+        inProgressSegment.write(element, this.eventTimeExtractor.getEventTime(element, context));
+    }
+
+    /**
+     * Creates {@link PinotSinkCommittable}s from elements received via {@link #write}
+     *
+     * @param flush Flush all currently known elements into the {@link PinotSinkCommittable}s
+     * @return List of {@link PinotSinkCommittable} to process in {@link org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter}
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkCommittable> prepareCommit(boolean flush) throws IOException {
+        List<PinotWriterSegment<IN>> segmentsToCommit = this.activeSegments.stream()
+                .filter(s -> flush || !s.acceptsElements())
+                .collect(Collectors.toList());
+        LOG.info("Identified {} segments to commit [subtaskId={}]", segmentsToCommit.size(), this.subtaskId);
+
+        LOG.info("Creating committables... [subtaskId={}]", subtaskId);
+        List<PinotSinkCommittable> committables = new ArrayList<>();
+        for (final PinotWriterSegment<IN> segment : segmentsToCommit) {
+            committables.add(segment.prepareCommit());
+        }
+        LOG.info("Created {} committables [subtaskId={}]", committables.size(), subtaskId);
+
+        this.activeSegments.removeAll(segmentsToCommit);
+        return committables;
+    }
+
+    /**
+     * Gets the {@link PinotWriterSegment} still accepting elements or creates a new one.
+     *
+     * @return {@link PinotWriterSegment} accepting at least one more element
+     */
+    private PinotWriterSegment<IN> getOrCreateInProgressSegment() {
+        final PinotWriterSegment<IN> latestSegment = Iterables.getLast(this.activeSegments, null);
+        if (latestSegment == null || !latestSegment.acceptsElements()) {
+            final PinotWriterSegment<IN> inProgressSegment = new PinotWriterSegment<>(this.maxRowsPerSegment, this.tempDirPrefix, this.jsonSerializer, this.fsAdapter);
+            this.activeSegments.add(inProgressSegment);
+            return inProgressSegment;
+        }
+        return latestSegment;
+    }
+
+    /**
+     * Snapshots the current state to be stored within a checkpoint. As we do not need to save any

Review comment:
       ```suggestion
        * As we do not need to save any
   ```

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        public CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    static class SegmentCommitter implements Callable<Boolean> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class);
+
+        final String pinotControllerHost;
+        final String pinotControllerPort;
+        final String tempDirPrefix;
+        final FileSystemAdapter fsAdapter;
+        final String dataFilePath;
+        final String segmentName;
+        final Schema tableSchema;
+        final TableConfig tableConfig;
+        final String timeColumnName;
+        final TimeUnit segmentTimeUnit;
+
+        /**
+         * @param pinotControllerHost Host of the Pinot controller
+         * @param pinotControllerPort Port of the Pinot controller
+         * @param fsAdapter           Filesystem adapter used to load data files from the shared file system
+         * @param dataFilePath        Data file to load from the shared file system
+         * @param segmentName         Name of the segment to create and commit
+         * @param tableSchema         Pinot table schema
+         * @param tableConfig         Pinot table config
+         * @param timeColumnName      Name of the column containing the timestamp
+         * @param segmentTimeUnit     Unit of the time column
+         */
+        public SegmentCommitter(String pinotControllerHost, String pinotControllerPort, String tempDirPrefix, FileSystemAdapter fsAdapter, String dataFilePath, String segmentName, Schema tableSchema, TableConfig tableConfig, String timeColumnName, TimeUnit segmentTimeUnit) {
+            this.pinotControllerHost = pinotControllerHost;
+            this.pinotControllerPort = pinotControllerPort;
+            this.tempDirPrefix = tempDirPrefix;
+            this.fsAdapter = fsAdapter;
+            this.dataFilePath = dataFilePath;
+            this.segmentName = segmentName;
+            this.tableSchema = tableSchema;
+            this.tableConfig = tableConfig;
+            this.timeColumnName = timeColumnName;
+            this.segmentTimeUnit = segmentTimeUnit;
+        }
+
+        /**
+         * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment
+         * and finally uploads the segment to the Pinot controller
+         *
+         * @return True if the commit succeeded
+         */
+        @Override
+        public Boolean call() {
+            try {
+                // Download data file from the shared filesystem
+                LOG.info("Downloading data file {} from shared file system...", dataFilePath);
+                File segmentData = fsAdapter.copyToLocalFile(dataFilePath);
+                LOG.info("Successfully downloaded data file {} from shared file system", dataFilePath);
+
+                File segmentFile = Files.createTempDirectory(this.tempDirPrefix).toFile();
+                LOG.info("Creating segment in " + segmentFile.getAbsolutePath());
+
+                // Creates a segment with name `segmentName` in `segmentFile`
+                this.generateSegment(segmentData, segmentFile, true);
+
+                // Uploads the recently created segment to the Pinot controller
+                this.uploadSegment(segmentFile);
+
+                // Commit successful
+                return true;
+            } catch (IOException e) {
+                LOG.error(e.getMessage());

Review comment:
       Provide a message with a bit more details, and error as second parameter.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import com.google.common.collect.Iterables;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Accepts incoming elements and creates {@link PinotSinkCommittable}s out of them on request.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotSinkWriter<IN> implements SinkWriter<IN, PinotSinkCommittable, Void> {
+
+    private static final Logger LOG = LoggerFactory.getLogger("PinotSinkWriter");
+
+    private final int maxRowsPerSegment;
+    private EventTimeExtractor<IN> eventTimeExtractor;
+    private final String tempDirPrefix;
+    private final JsonSerializer<IN> jsonSerializer;
+
+    private final List<PinotWriterSegment<IN>> activeSegments;
+    private final FileSystemAdapter fsAdapter;
+
+    private final int subtaskId;
+
+    /**
+     * @param subtaskId          Subtask id provided by Flink
+     * @param maxRowsPerSegment  Maximum number of rows to be stored within a Pinot segment
+     * @param eventTimeExtractor Defines the way event times are extracted from received objects
+     * @param tempDirPrefix      Prefix for temp directories used
+     * @param jsonSerializer     Serializer used to convert elements to JSON
+     * @param fsAdapter          Filesystem adapter used to save files for sharing files across nodes
+     */
+    public PinotSinkWriter(int subtaskId, int maxRowsPerSegment, EventTimeExtractor<IN> eventTimeExtractor, String tempDirPrefix, JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        this.subtaskId = subtaskId;
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.eventTimeExtractor = checkNotNull(eventTimeExtractor);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.activeSegments = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements from an upstream tasks and writes them into {@link PinotWriterSegment}
+     *
+     * @param element Object from upstream task
+     * @param context SinkWriter context
+     * @throws IOException
+     */
+    @Override
+    public void write(IN element, Context context) throws IOException {
+        final PinotWriterSegment<IN> inProgressSegment = this.getOrCreateInProgressSegment();
+        inProgressSegment.write(element, this.eventTimeExtractor.getEventTime(element, context));
+    }
+
+    /**
+     * Creates {@link PinotSinkCommittable}s from elements received via {@link #write}

Review comment:
       A bit more information what this method is doing in respect to the activeSegments would be great.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import com.google.common.collect.Iterables;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Accepts incoming elements and creates {@link PinotSinkCommittable}s out of them on request.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotSinkWriter<IN> implements SinkWriter<IN, PinotSinkCommittable, Void> {
+
+    private static final Logger LOG = LoggerFactory.getLogger("PinotSinkWriter");
+
+    private final int maxRowsPerSegment;
+    private EventTimeExtractor<IN> eventTimeExtractor;
+    private final String tempDirPrefix;
+    private final JsonSerializer<IN> jsonSerializer;
+
+    private final List<PinotWriterSegment<IN>> activeSegments;
+    private final FileSystemAdapter fsAdapter;
+
+    private final int subtaskId;
+
+    /**
+     * @param subtaskId          Subtask id provided by Flink
+     * @param maxRowsPerSegment  Maximum number of rows to be stored within a Pinot segment
+     * @param eventTimeExtractor Defines the way event times are extracted from received objects
+     * @param tempDirPrefix      Prefix for temp directories used
+     * @param jsonSerializer     Serializer used to convert elements to JSON
+     * @param fsAdapter          Filesystem adapter used to save files for sharing files across nodes
+     */
+    public PinotSinkWriter(int subtaskId, int maxRowsPerSegment, EventTimeExtractor<IN> eventTimeExtractor, String tempDirPrefix, JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        this.subtaskId = subtaskId;
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.eventTimeExtractor = checkNotNull(eventTimeExtractor);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.activeSegments = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements from an upstream tasks and writes them into {@link PinotWriterSegment}
+     *
+     * @param element Object from upstream task
+     * @param context SinkWriter context
+     * @throws IOException
+     */
+    @Override
+    public void write(IN element, Context context) throws IOException {
+        final PinotWriterSegment<IN> inProgressSegment = this.getOrCreateInProgressSegment();
+        inProgressSegment.write(element, this.eventTimeExtractor.getEventTime(element, context));
+    }
+
+    /**
+     * Creates {@link PinotSinkCommittable}s from elements received via {@link #write}
+     *
+     * @param flush Flush all currently known elements into the {@link PinotSinkCommittable}s
+     * @return List of {@link PinotSinkCommittable} to process in {@link org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter}
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkCommittable> prepareCommit(boolean flush) throws IOException {

Review comment:
       Again switch to debug logging.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        public CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    static class SegmentCommitter implements Callable<Boolean> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class);
+
+        final String pinotControllerHost;
+        final String pinotControllerPort;
+        final String tempDirPrefix;
+        final FileSystemAdapter fsAdapter;
+        final String dataFilePath;
+        final String segmentName;
+        final Schema tableSchema;
+        final TableConfig tableConfig;
+        final String timeColumnName;
+        final TimeUnit segmentTimeUnit;
+
+        /**
+         * @param pinotControllerHost Host of the Pinot controller
+         * @param pinotControllerPort Port of the Pinot controller
+         * @param fsAdapter           Filesystem adapter used to load data files from the shared file system
+         * @param dataFilePath        Data file to load from the shared file system
+         * @param segmentName         Name of the segment to create and commit
+         * @param tableSchema         Pinot table schema
+         * @param tableConfig         Pinot table config
+         * @param timeColumnName      Name of the column containing the timestamp
+         * @param segmentTimeUnit     Unit of the time column
+         */
+        public SegmentCommitter(String pinotControllerHost, String pinotControllerPort, String tempDirPrefix, FileSystemAdapter fsAdapter, String dataFilePath, String segmentName, Schema tableSchema, TableConfig tableConfig, String timeColumnName, TimeUnit segmentTimeUnit) {
+            this.pinotControllerHost = pinotControllerHost;
+            this.pinotControllerPort = pinotControllerPort;
+            this.tempDirPrefix = tempDirPrefix;
+            this.fsAdapter = fsAdapter;
+            this.dataFilePath = dataFilePath;
+            this.segmentName = segmentName;
+            this.tableSchema = tableSchema;
+            this.tableConfig = tableConfig;
+            this.timeColumnName = timeColumnName;
+            this.segmentTimeUnit = segmentTimeUnit;
+        }
+
+        /**
+         * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment
+         * and finally uploads the segment to the Pinot controller
+         *
+         * @return True if the commit succeeded
+         */
+        @Override
+        public Boolean call() {
+            try {
+                // Download data file from the shared filesystem
+                LOG.info("Downloading data file {} from shared file system...", dataFilePath);
+                File segmentData = fsAdapter.copyToLocalFile(dataFilePath);
+                LOG.info("Successfully downloaded data file {} from shared file system", dataFilePath);
+
+                File segmentFile = Files.createTempDirectory(this.tempDirPrefix).toFile();
+                LOG.info("Creating segment in " + segmentFile.getAbsolutePath());
+
+                // Creates a segment with name `segmentName` in `segmentFile`
+                this.generateSegment(segmentData, segmentFile, true);
+
+                // Uploads the recently created segment to the Pinot controller
+                this.uploadSegment(segmentFile);
+
+                // Commit successful
+                return true;
+            } catch (IOException e) {
+                LOG.error(e.getMessage());
+
+                // Commit failed
+                return false;
+            }
+        }
+
+        /**
+         * Creates a segment from the given parameters.
+         * This method was adapted from {@link org.apache.pinot.tools.admin.command.CreateSegmentCommand}.
+         *
+         * @param dataFile                  File containing the JSON data
+         * @param outDir                    Segment target path
+         * @param _postCreationVerification Verify segment after generation
+         */
+        public void generateSegment(File dataFile, File outDir, Boolean _postCreationVerification) {
+            SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(tableConfig, tableSchema);
+            segmentGeneratorConfig.setSegmentName(segmentName);
+            segmentGeneratorConfig.setSegmentTimeUnit(segmentTimeUnit);
+            segmentGeneratorConfig.setTimeColumnName(timeColumnName);
+            segmentGeneratorConfig.setInputFilePath(dataFile.getPath());
+            segmentGeneratorConfig.setFormat(FileFormat.JSON);
+            segmentGeneratorConfig.setOutDir(outDir.getPath());
+            segmentGeneratorConfig.setTableName(tableConfig.getTableName());
+
+            try {
+                SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+                driver.init(segmentGeneratorConfig);
+                driver.build();
+                File indexDir = new File(outDir, segmentName);
+                LOG.info("Successfully created segment: {} in directory: {}", segmentName, indexDir);
+                if (_postCreationVerification) {
+                    LOG.info("Verifying the segment by loading it");
+                    ImmutableSegment segment = ImmutableSegmentLoader.load(indexDir, ReadMode.mmap);
+                    LOG.info("Successfully loaded segment: {} of size: {} bytes", segmentName,
+                            segment.getSegmentSizeBytes());
+                    segment.destroy();
+                }
+            } catch (Exception e) {
+                LOG.error(e.getMessage());
+                throw new RuntimeException("Caught exception while generating segment from file: " + dataFile.getPath());
+            }
+            LOG.info("Successfully created 1 segment from data file: {}", dataFile);
+        }
+
+        /**
+         * Uploads a segment using the Pinot admin tool.
+         *
+         * @param segmentFile File containing the segment to upload
+         * @throws IOException
+         */
+        public void uploadSegment(File segmentFile) throws IOException {

Review comment:
       private?

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import com.google.common.collect.Iterables;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Accepts incoming elements and creates {@link PinotSinkCommittable}s out of them on request.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotSinkWriter<IN> implements SinkWriter<IN, PinotSinkCommittable, Void> {
+
+    private static final Logger LOG = LoggerFactory.getLogger("PinotSinkWriter");
+
+    private final int maxRowsPerSegment;
+    private EventTimeExtractor<IN> eventTimeExtractor;
+    private final String tempDirPrefix;
+    private final JsonSerializer<IN> jsonSerializer;
+
+    private final List<PinotWriterSegment<IN>> activeSegments;
+    private final FileSystemAdapter fsAdapter;
+
+    private final int subtaskId;
+
+    /**
+     * @param subtaskId          Subtask id provided by Flink
+     * @param maxRowsPerSegment  Maximum number of rows to be stored within a Pinot segment
+     * @param eventTimeExtractor Defines the way event times are extracted from received objects
+     * @param tempDirPrefix      Prefix for temp directories used
+     * @param jsonSerializer     Serializer used to convert elements to JSON
+     * @param fsAdapter          Filesystem adapter used to save files for sharing files across nodes
+     */
+    public PinotSinkWriter(int subtaskId, int maxRowsPerSegment, EventTimeExtractor<IN> eventTimeExtractor, String tempDirPrefix, JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        this.subtaskId = subtaskId;
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.eventTimeExtractor = checkNotNull(eventTimeExtractor);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.activeSegments = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements from an upstream tasks and writes them into {@link PinotWriterSegment}
+     *
+     * @param element Object from upstream task
+     * @param context SinkWriter context
+     * @throws IOException
+     */
+    @Override
+    public void write(IN element, Context context) throws IOException {
+        final PinotWriterSegment<IN> inProgressSegment = this.getOrCreateInProgressSegment();
+        inProgressSegment.write(element, this.eventTimeExtractor.getEventTime(element, context));
+    }
+
+    /**
+     * Creates {@link PinotSinkCommittable}s from elements received via {@link #write}
+     *
+     * @param flush Flush all currently known elements into the {@link PinotSinkCommittable}s
+     * @return List of {@link PinotSinkCommittable} to process in {@link org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter}
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkCommittable> prepareCommit(boolean flush) throws IOException {
+        List<PinotWriterSegment<IN>> segmentsToCommit = this.activeSegments.stream()
+                .filter(s -> flush || !s.acceptsElements())
+                .collect(Collectors.toList());
+        LOG.info("Identified {} segments to commit [subtaskId={}]", segmentsToCommit.size(), this.subtaskId);
+
+        LOG.info("Creating committables... [subtaskId={}]", subtaskId);
+        List<PinotSinkCommittable> committables = new ArrayList<>();
+        for (final PinotWriterSegment<IN> segment : segmentsToCommit) {
+            committables.add(segment.prepareCommit());
+        }
+        LOG.info("Created {} committables [subtaskId={}]", committables.size(), subtaskId);
+
+        this.activeSegments.removeAll(segmentsToCommit);
+        return committables;
+    }
+
+    /**
+     * Gets the {@link PinotWriterSegment} still accepting elements or creates a new one.
+     *
+     * @return {@link PinotWriterSegment} accepting at least one more element
+     */
+    private PinotWriterSegment<IN> getOrCreateInProgressSegment() {
+        final PinotWriterSegment<IN> latestSegment = Iterables.getLast(this.activeSegments, null);
+        if (latestSegment == null || !latestSegment.acceptsElements()) {
+            final PinotWriterSegment<IN> inProgressSegment = new PinotWriterSegment<>(this.maxRowsPerSegment, this.tempDirPrefix, this.jsonSerializer, this.fsAdapter);
+            this.activeSegments.add(inProgressSegment);
+            return inProgressSegment;
+        }
+        return latestSegment;
+    }
+
+    /**
+     * Snapshots the current state to be stored within a checkpoint. As we do not need to save any
+     * information in snapshots, this method always returns an empty ArrayList.
+     *
+     * @return always an empty ArrayList
+     */
+    @Override
+    public List<Void> snapshotState() {

Review comment:
       What happens with the temporary written files?

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/EmulatedPinotSinkTest.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.filesystem.LocalFileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.pinot.client.ResultSet;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * E2e tests for Pinot Sink using BATCH and STREAMING execution mode
+ */
+public class EmulatedPinotSinkTest extends PinotTestBase {
+
+    /**
+     * Tests the BATCH execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testBatchSink() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.BATCH);
+        env.setParallelism(2);
+
+        List<SingleColumnTableRow> data = getTestData(12);
+        this.setupDataStream(env, data);
+
+        // Run
+        env.execute();
+
+        TimeUnit.SECONDS.sleep(5);
+
+        checkForDataInPinot(data, data.size());
+    }
+
+    /**
+     * Tests the STREAMING execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testStreamingSink() throws Exception {
+        final Configuration conf = new Configuration();
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(conf);
+        env.setRuntimeMode(RuntimeExecutionMode.STREAMING);
+        env.setParallelism(2);
+        env.enableCheckpointing(50);
+
+        List<SingleColumnTableRow> data = getTestData(1000);
+        this.setupDataStream(env, data);
+
+        // Run
+        env.execute();
+
+        // Wait until the checkpoint was created and the segments were committed by the GlobalCommitter
+        TimeUnit.SECONDS.sleep(5);

Review comment:
       Maybe my suggestion for batch execution also works here because the streaming mode is finite when using the `NumberSequenceSource`

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/EmulatedPinotSinkTest.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.filesystem.LocalFileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.pinot.client.ResultSet;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * E2e tests for Pinot Sink using BATCH and STREAMING execution mode
+ */
+public class EmulatedPinotSinkTest extends PinotTestBase {
+
+    /**
+     * Tests the BATCH execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testBatchSink() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.BATCH);
+        env.setParallelism(2);
+
+        List<SingleColumnTableRow> data = getTestData(12);
+        this.setupDataStream(env, data);
+
+        // Run
+        env.execute();
+
+        TimeUnit.SECONDS.sleep(5);

Review comment:
       I do not think you need a timeout for the batch mode because the job should go eventually into finished state.
   You can do something along the lines of https://github.com/apache/flink/blob/bebf3b5a105dd4bc21882116570c6d71299269a6/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/FileSinkITBase.java#L80

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/EmulatedPinotSinkTest.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.filesystem.LocalFileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.pinot.client.ResultSet;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * E2e tests for Pinot Sink using BATCH and STREAMING execution mode
+ */
+public class EmulatedPinotSinkTest extends PinotTestBase {

Review comment:
       ```suggestion
   public class PinotSinkTest extends PinotTestBase {
   ```

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.util.TestLogger;
+import org.apache.pinot.spi.config.table.*;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.Wait;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Base class for PinotSink e2e tests
+ */
+@Testcontainers
+public class PinotTestBase extends TestLogger implements Serializable {
+
+    public static final String DOCKER_IMAGE_NAME = "apachepinot/pinot:0.6.0";
+    public static final Integer PINOT_INTERNAL_BROKER_PORT = 8000;
+    public static final Integer PINOT_INTERNAL_CONTROLLER_PORT = 9000;
+
+    protected static final TableConfig TABLE_CONFIG = PinotTableConfig.getTableConfig();
+    protected static final String TABLE_NAME = TABLE_CONFIG.getTableName();
+    protected static final Schema TABLE_SCHEMA = PinotTableConfig.getTableSchema();
+    protected static PinotTestHelper pinotHelper;
+
+    /**
+     * Creates the Pinot testcontainer. We delay the start of tests until Pinot has started all
+     * internal components. This is identified through a log statement.
+     */
+    @Container
+    public GenericContainer<?> pinot = new GenericContainer<>(DockerImageName.parse(DOCKER_IMAGE_NAME))
+            .withCommand("QuickStart", "-type", "batch")
+            .withExposedPorts(PINOT_INTERNAL_BROKER_PORT, PINOT_INTERNAL_CONTROLLER_PORT)
+            .waitingFor(Wait
+                    .forLogMessage(".*You can always go to http://localhost:9000 to play around in the query console.*\\n", 1)
+                    // Allow Pinot to take up to 90s for starting up
+                    .withStartupTimeout(Duration.ofSeconds(90))
+            );
+
+    /**
+     * Creates a new instance of the {@link PinotTestHelper} using the testcontainer port mappings
+     * and creates the test table.
+     *
+     * @throws IOException
+     */
+    @BeforeEach
+    public void setUp() throws IOException {
+        pinotHelper = new PinotTestHelper(getPinotHost(), getPinotControllerPort(), getPinotBrokerPort());
+        pinotHelper.createTable(TABLE_CONFIG, TABLE_SCHEMA);
+    }
+
+    /**
+     * Delete the test table after each test.
+     *
+     * @throws Exception
+     */
+    @AfterEach
+    public void tearDown() throws Exception {
+        pinotHelper.deleteTable(TABLE_CONFIG, TABLE_SCHEMA);
+    }
+
+    /**
+     * Returns the host the Pinot container is available at
+     *
+     * @return Pinot container host
+     */
+    public String getPinotHost() {

Review comment:
       protected?

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.util.TestLogger;
+import org.apache.pinot.spi.config.table.*;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.Wait;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Base class for PinotSink e2e tests
+ */
+@Testcontainers
+public class PinotTestBase extends TestLogger implements Serializable {
+
+    public static final String DOCKER_IMAGE_NAME = "apachepinot/pinot:0.6.0";
+    public static final Integer PINOT_INTERNAL_BROKER_PORT = 8000;
+    public static final Integer PINOT_INTERNAL_CONTROLLER_PORT = 9000;

Review comment:
       private?

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.util.TestLogger;
+import org.apache.pinot.spi.config.table.*;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.Wait;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Base class for PinotSink e2e tests
+ */
+@Testcontainers
+public class PinotTestBase extends TestLogger implements Serializable {
+
+    public static final String DOCKER_IMAGE_NAME = "apachepinot/pinot:0.6.0";
+    public static final Integer PINOT_INTERNAL_BROKER_PORT = 8000;
+    public static final Integer PINOT_INTERNAL_CONTROLLER_PORT = 9000;
+
+    protected static final TableConfig TABLE_CONFIG = PinotTableConfig.getTableConfig();
+    protected static final String TABLE_NAME = TABLE_CONFIG.getTableName();
+    protected static final Schema TABLE_SCHEMA = PinotTableConfig.getTableSchema();
+    protected static PinotTestHelper pinotHelper;
+
+    /**
+     * Creates the Pinot testcontainer. We delay the start of tests until Pinot has started all
+     * internal components. This is identified through a log statement.
+     */
+    @Container
+    public GenericContainer<?> pinot = new GenericContainer<>(DockerImageName.parse(DOCKER_IMAGE_NAME))
+            .withCommand("QuickStart", "-type", "batch")
+            .withExposedPorts(PINOT_INTERNAL_BROKER_PORT, PINOT_INTERNAL_CONTROLLER_PORT)
+            .waitingFor(Wait
+                    .forLogMessage(".*You can always go to http://localhost:9000 to play around in the query console.*\\n", 1)
+                    // Allow Pinot to take up to 90s for starting up
+                    .withStartupTimeout(Duration.ofSeconds(90))
+            );
+
+    /**
+     * Creates a new instance of the {@link PinotTestHelper} using the testcontainer port mappings
+     * and creates the test table.
+     *
+     * @throws IOException
+     */
+    @BeforeEach
+    public void setUp() throws IOException {
+        pinotHelper = new PinotTestHelper(getPinotHost(), getPinotControllerPort(), getPinotBrokerPort());
+        pinotHelper.createTable(TABLE_CONFIG, TABLE_SCHEMA);
+    }
+
+    /**
+     * Delete the test table after each test.
+     *
+     * @throws Exception
+     */
+    @AfterEach
+    public void tearDown() throws Exception {
+        pinotHelper.deleteTable(TABLE_CONFIG, TABLE_SCHEMA);
+    }
+
+    /**
+     * Returns the host the Pinot container is available at
+     *
+     * @return Pinot container host
+     */
+    public String getPinotHost() {
+        return this.pinot.getHost();
+    }
+
+
+    /**
+     * Returns the Pinot controller port from the container ports.
+     *
+     * @return Pinot controller port
+     */
+    public String getPinotControllerPort() {
+        return this.pinot.getMappedPort(PINOT_INTERNAL_CONTROLLER_PORT).toString();
+    }
+
+    /**
+     * Returns the Pinot broker port from the container ports.
+     *
+     * @return Pinot broker port
+     */
+    public String getPinotBrokerPort() {

Review comment:
       protected?

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.util.TestLogger;
+import org.apache.pinot.spi.config.table.*;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.Wait;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Base class for PinotSink e2e tests
+ */
+@Testcontainers
+public class PinotTestBase extends TestLogger implements Serializable {
+
+    public static final String DOCKER_IMAGE_NAME = "apachepinot/pinot:0.6.0";
+    public static final Integer PINOT_INTERNAL_BROKER_PORT = 8000;
+    public static final Integer PINOT_INTERNAL_CONTROLLER_PORT = 9000;
+
+    protected static final TableConfig TABLE_CONFIG = PinotTableConfig.getTableConfig();
+    protected static final String TABLE_NAME = TABLE_CONFIG.getTableName();
+    protected static final Schema TABLE_SCHEMA = PinotTableConfig.getTableSchema();
+    protected static PinotTestHelper pinotHelper;
+
+    /**
+     * Creates the Pinot testcontainer. We delay the start of tests until Pinot has started all
+     * internal components. This is identified through a log statement.
+     */
+    @Container
+    public GenericContainer<?> pinot = new GenericContainer<>(DockerImageName.parse(DOCKER_IMAGE_NAME))
+            .withCommand("QuickStart", "-type", "batch")
+            .withExposedPorts(PINOT_INTERNAL_BROKER_PORT, PINOT_INTERNAL_CONTROLLER_PORT)
+            .waitingFor(Wait
+                    .forLogMessage(".*You can always go to http://localhost:9000 to play around in the query console.*\\n", 1)

Review comment:
       Nit: Maybe wait until the port is consumable? When pinot is updated in the future this check might be hard to fix.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.HttpEntity;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHost;
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        this.controllerHost = checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+
+            String body = EntityUtils.toString(response.getEntity());

Review comment:
       @mschroederi I am also interested what happens in the error case. In general it should be safe to fail the Flink job if the connection to pinot cannot be established.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotWriterSegment.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@link PinotWriterSegment} represents exactly one segment that can be found in the Pinot
+ * cluster once the commit has been completed.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotWriterSegment<IN> implements Serializable {
+
+    private static final Logger LOG = LoggerFactory.getLogger("PinotWriterSegment");
+
+    private final int maxRowsPerSegment;
+    private final String tempDirPrefix;
+    private final JsonSerializer<IN> jsonSerializer;
+    private final FileSystemAdapter fsAdapter;
+
+    private boolean acceptsElements = true;
+
+    private final List<IN> elements;
+    private File dataFile;
+    private long minTimestamp = Long.MAX_VALUE;
+    private long maxTimestamp = Long.MIN_VALUE;
+
+    /**
+     * @param maxRowsPerSegment Maximum number of rows to be stored within a Pinot segment
+     * @param tempDirPrefix     Prefix for temp directories used
+     * @param jsonSerializer    Serializer used to convert elements to JSON
+     * @param fsAdapter         Filesystem adapter used to save files for sharing files across nodes
+     */
+    protected PinotWriterSegment(int maxRowsPerSegment, String tempDirPrefix, JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        checkArgument(maxRowsPerSegment > 0L);
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.elements = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements and stores them in memory until either {@link #maxRowsPerSegment} is reached
+     * or {@link #prepareCommit} is called.
+     *
+     * @param element   Object from upstream task
+     * @param timestamp Timestamp assigned to element
+     * @throws IOException
+     */
+    public void write(IN element, long timestamp) throws IOException {
+        if (!this.acceptsElements()) {
+            throw new IllegalStateException("This PinotSegmentWriter does not accept any elements anymore.");
+        }
+        this.elements.add(element);
+        this.minTimestamp = Long.min(this.minTimestamp, timestamp);
+        this.maxTimestamp = Long.max(this.maxTimestamp, timestamp);
+
+        // Writes elements to local filesystem once the maximum number of items is reached
+        if (this.elements.size() == this.maxRowsPerSegment) {

Review comment:
       I think it is a good idea to directly write to the shared fs rather than having the local copy. Do you plan to implement it in this PR (extending the `FileSystemAdapater` interface should not be to complicated) or leave it as future optimization?

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.util.TestLogger;
+import org.apache.pinot.spi.config.table.*;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.Wait;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Base class for PinotSink e2e tests
+ */
+@Testcontainers
+public class PinotTestBase extends TestLogger implements Serializable {
+
+    public static final String DOCKER_IMAGE_NAME = "apachepinot/pinot:0.6.0";
+    public static final Integer PINOT_INTERNAL_BROKER_PORT = 8000;
+    public static final Integer PINOT_INTERNAL_CONTROLLER_PORT = 9000;
+
+    protected static final TableConfig TABLE_CONFIG = PinotTableConfig.getTableConfig();
+    protected static final String TABLE_NAME = TABLE_CONFIG.getTableName();
+    protected static final Schema TABLE_SCHEMA = PinotTableConfig.getTableSchema();
+    protected static PinotTestHelper pinotHelper;
+
+    /**
+     * Creates the Pinot testcontainer. We delay the start of tests until Pinot has started all
+     * internal components. This is identified through a log statement.
+     */
+    @Container
+    public GenericContainer<?> pinot = new GenericContainer<>(DockerImageName.parse(DOCKER_IMAGE_NAME))
+            .withCommand("QuickStart", "-type", "batch")
+            .withExposedPorts(PINOT_INTERNAL_BROKER_PORT, PINOT_INTERNAL_CONTROLLER_PORT)
+            .waitingFor(Wait
+                    .forLogMessage(".*You can always go to http://localhost:9000 to play around in the query console.*\\n", 1)
+                    // Allow Pinot to take up to 90s for starting up
+                    .withStartupTimeout(Duration.ofSeconds(90))
+            );
+
+    /**
+     * Creates a new instance of the {@link PinotTestHelper} using the testcontainer port mappings
+     * and creates the test table.
+     *
+     * @throws IOException
+     */
+    @BeforeEach
+    public void setUp() throws IOException {
+        pinotHelper = new PinotTestHelper(getPinotHost(), getPinotControllerPort(), getPinotBrokerPort());
+        pinotHelper.createTable(TABLE_CONFIG, TABLE_SCHEMA);
+    }
+
+    /**
+     * Delete the test table after each test.
+     *
+     * @throws Exception
+     */
+    @AfterEach
+    public void tearDown() throws Exception {
+        pinotHelper.deleteTable(TABLE_CONFIG, TABLE_SCHEMA);
+    }
+
+    /**
+     * Returns the host the Pinot container is available at
+     *
+     * @return Pinot container host
+     */
+    public String getPinotHost() {
+        return this.pinot.getHost();
+    }
+
+
+    /**
+     * Returns the Pinot controller port from the container ports.
+     *
+     * @return Pinot controller port
+     */
+    public String getPinotControllerPort() {
+        return this.pinot.getMappedPort(PINOT_INTERNAL_CONTROLLER_PORT).toString();
+    }
+
+    /**
+     * Returns the Pinot broker port from the container ports.
+     *
+     * @return Pinot broker port
+     */
+    public String getPinotBrokerPort() {
+        return this.pinot.getMappedPort(PINOT_INTERNAL_BROKER_PORT).toString();
+    }
+
+    /**
+     * Class defining the elements passed to the {@link PinotSink} during the tests.
+     */
+    static class SingleColumnTableRow {
+
+        private String _col1;
+        private Long _timestamp;
+
+        SingleColumnTableRow(@JsonProperty(value = "col1", required = true) String col1,
+                             @JsonProperty(value = "timestamp", required = true) Long timestamp) {
+            this._col1 = col1;
+            this._timestamp = timestamp;
+        }
+
+        @JsonProperty("col1")
+        public String getCol1() {
+            return this._col1;
+        }
+
+        public void setCol1(String _col1) {
+            this._col1 = _col1;
+        }
+
+        @JsonProperty("timestamp")
+        public Long getTimestamp() {
+            return this._timestamp;
+        }
+
+        public void setTimestamp(Long timestamp) {
+            this._timestamp = timestamp;
+        }
+    }
+
+
+    /**
+     * EventTimeExtractor for {@link SingleColumnTableRow} used in e2e tests.
+     * Extracts the timestamp column from {@link SingleColumnTableRow}.
+     */
+    static class SingleColumnTableRowEventTimeExtractor extends EventTimeExtractor<SingleColumnTableRow> {
+
+        @Override
+        public long getEventTime(SingleColumnTableRow element, SinkWriter.Context context) {
+            return element.getTimestamp();
+        }
+
+        @Override
+        public String getTimeColumn() {
+            return "timestamp";
+        }
+
+        @Override
+        public TimeUnit getSegmentTimeUnit() {
+            return TimeUnit.MILLISECONDS;
+        }
+    }
+
+    /**
+     * Serializes {@link SingleColumnTableRow} to JSON.
+     */
+    static class SingleColumnTableRowSerializer extends JsonSerializer<SingleColumnTableRow> {
+
+        @Override
+        public String toJson(SingleColumnTableRow element) {
+            return JsonUtils.objectToJsonNode(element).toString();
+        }
+    }
+
+    /**
+     * Pinot table configuration helpers.
+     */
+    static class PinotTableConfig {

Review comment:
       private?

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.util.TestLogger;
+import org.apache.pinot.spi.config.table.*;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.Wait;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Base class for PinotSink e2e tests
+ */
+@Testcontainers
+public class PinotTestBase extends TestLogger implements Serializable {
+
+    public static final String DOCKER_IMAGE_NAME = "apachepinot/pinot:0.6.0";
+    public static final Integer PINOT_INTERNAL_BROKER_PORT = 8000;
+    public static final Integer PINOT_INTERNAL_CONTROLLER_PORT = 9000;
+
+    protected static final TableConfig TABLE_CONFIG = PinotTableConfig.getTableConfig();
+    protected static final String TABLE_NAME = TABLE_CONFIG.getTableName();
+    protected static final Schema TABLE_SCHEMA = PinotTableConfig.getTableSchema();
+    protected static PinotTestHelper pinotHelper;
+
+    /**
+     * Creates the Pinot testcontainer. We delay the start of tests until Pinot has started all
+     * internal components. This is identified through a log statement.
+     */
+    @Container
+    public GenericContainer<?> pinot = new GenericContainer<>(DockerImageName.parse(DOCKER_IMAGE_NAME))
+            .withCommand("QuickStart", "-type", "batch")
+            .withExposedPorts(PINOT_INTERNAL_BROKER_PORT, PINOT_INTERNAL_CONTROLLER_PORT)
+            .waitingFor(Wait
+                    .forLogMessage(".*You can always go to http://localhost:9000 to play around in the query console.*\\n", 1)
+                    // Allow Pinot to take up to 90s for starting up
+                    .withStartupTimeout(Duration.ofSeconds(90))
+            );
+
+    /**
+     * Creates a new instance of the {@link PinotTestHelper} using the testcontainer port mappings
+     * and creates the test table.
+     *
+     * @throws IOException
+     */
+    @BeforeEach
+    public void setUp() throws IOException {
+        pinotHelper = new PinotTestHelper(getPinotHost(), getPinotControllerPort(), getPinotBrokerPort());
+        pinotHelper.createTable(TABLE_CONFIG, TABLE_SCHEMA);
+    }
+
+    /**
+     * Delete the test table after each test.
+     *
+     * @throws Exception
+     */
+    @AfterEach
+    public void tearDown() throws Exception {
+        pinotHelper.deleteTable(TABLE_CONFIG, TABLE_SCHEMA);
+    }
+
+    /**
+     * Returns the host the Pinot container is available at
+     *
+     * @return Pinot container host
+     */
+    public String getPinotHost() {
+        return this.pinot.getHost();
+    }
+
+
+    /**
+     * Returns the Pinot controller port from the container ports.
+     *
+     * @return Pinot controller port
+     */
+    public String getPinotControllerPort() {

Review comment:
       protected?

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);

Review comment:
       Similar with TableConfig




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594298275



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        public CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    static class SegmentCommitter implements Callable<Boolean> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class);
+
+        final String pinotControllerHost;
+        final String pinotControllerPort;
+        final String tempDirPrefix;
+        final FileSystemAdapter fsAdapter;
+        final String dataFilePath;
+        final String segmentName;
+        final Schema tableSchema;
+        final TableConfig tableConfig;
+        final String timeColumnName;
+        final TimeUnit segmentTimeUnit;
+
+        /**
+         * @param pinotControllerHost Host of the Pinot controller
+         * @param pinotControllerPort Port of the Pinot controller
+         * @param fsAdapter           Filesystem adapter used to load data files from the shared file system
+         * @param dataFilePath        Data file to load from the shared file system
+         * @param segmentName         Name of the segment to create and commit
+         * @param tableSchema         Pinot table schema
+         * @param tableConfig         Pinot table config
+         * @param timeColumnName      Name of the column containing the timestamp
+         * @param segmentTimeUnit     Unit of the time column
+         */
+        public SegmentCommitter(String pinotControllerHost, String pinotControllerPort, String tempDirPrefix, FileSystemAdapter fsAdapter, String dataFilePath, String segmentName, Schema tableSchema, TableConfig tableConfig, String timeColumnName, TimeUnit segmentTimeUnit) {
+            this.pinotControllerHost = pinotControllerHost;
+            this.pinotControllerPort = pinotControllerPort;
+            this.tempDirPrefix = tempDirPrefix;
+            this.fsAdapter = fsAdapter;
+            this.dataFilePath = dataFilePath;
+            this.segmentName = segmentName;
+            this.tableSchema = tableSchema;
+            this.tableConfig = tableConfig;
+            this.timeColumnName = timeColumnName;
+            this.segmentTimeUnit = segmentTimeUnit;
+        }
+
+        /**
+         * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment
+         * and finally uploads the segment to the Pinot controller
+         *
+         * @return True if the commit succeeded
+         */
+        @Override
+        public Boolean call() {
+            try {
+                // Download data file from the shared filesystem
+                LOG.info("Downloading data file {} from shared file system...", dataFilePath);
+                File segmentData = fsAdapter.copyToLocalFile(dataFilePath);
+                LOG.info("Successfully downloaded data file {} from shared file system", dataFilePath);
+
+                File segmentFile = Files.createTempDirectory(this.tempDirPrefix).toFile();
+                LOG.info("Creating segment in " + segmentFile.getAbsolutePath());
+
+                // Creates a segment with name `segmentName` in `segmentFile`
+                this.generateSegment(segmentData, segmentFile, true);
+
+                // Uploads the recently created segment to the Pinot controller
+                this.uploadSegment(segmentFile);
+
+                // Commit successful
+                return true;
+            } catch (IOException e) {
+                LOG.error(e.getMessage());
+
+                // Commit failed
+                return false;
+            }
+        }
+
+        /**
+         * Creates a segment from the given parameters.
+         * This method was adapted from {@link org.apache.pinot.tools.admin.command.CreateSegmentCommand}.
+         *
+         * @param dataFile                  File containing the JSON data
+         * @param outDir                    Segment target path
+         * @param _postCreationVerification Verify segment after generation
+         */
+        public void generateSegment(File dataFile, File outDir, Boolean _postCreationVerification) {
+            SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(tableConfig, tableSchema);
+            segmentGeneratorConfig.setSegmentName(segmentName);
+            segmentGeneratorConfig.setSegmentTimeUnit(segmentTimeUnit);
+            segmentGeneratorConfig.setTimeColumnName(timeColumnName);
+            segmentGeneratorConfig.setInputFilePath(dataFile.getPath());
+            segmentGeneratorConfig.setFormat(FileFormat.JSON);
+            segmentGeneratorConfig.setOutDir(outDir.getPath());
+            segmentGeneratorConfig.setTableName(tableConfig.getTableName());
+
+            try {
+                SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+                driver.init(segmentGeneratorConfig);
+                driver.build();
+                File indexDir = new File(outDir, segmentName);
+                LOG.info("Successfully created segment: {} in directory: {}", segmentName, indexDir);
+                if (_postCreationVerification) {
+                    LOG.info("Verifying the segment by loading it");
+                    ImmutableSegment segment = ImmutableSegmentLoader.load(indexDir, ReadMode.mmap);
+                    LOG.info("Successfully loaded segment: {} of size: {} bytes", segmentName,
+                            segment.getSegmentSizeBytes());
+                    segment.destroy();
+                }
+            } catch (Exception e) {

Review comment:
       The `SegmentIndexCreationDriverImpl` (`init` and `build`) and the `ImmutableSegmentLoader` (`load`) both throw generic Exceptions, so we have to catch those generic Exceptions here.
   I've added a comment to explain this




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r596171697



##########
File path: flink-connector-pinot/pom.xml
##########
@@ -0,0 +1,225 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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/maven-v4_0_0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.bahir</groupId>
+        <artifactId>bahir-flink-parent_2.11</artifactId>
+        <version>1.1-SNAPSHOT</version>
+        <relativePath>..</relativePath>
+    </parent>
+
+    <artifactId>flink-connector-pinot_2.11</artifactId>
+    <name>flink-connector-pinot</name>
+
+
+    <packaging>jar</packaging>
+
+    <!-- Allow users to pass custom connector versions -->
+    <properties>
+        <pinot.version>0.6.0</pinot.version>
+
+        <testcontainers.version>1.15.2</testcontainers.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-scala_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+        </dependency>
+
+        <!-- This is used to interact with Pinot Controller -->
+        <dependency>
+            <groupId>org.apache.pinot</groupId>
+            <artifactId>pinot-tools</artifactId>
+            <version>${pinot.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpmime</artifactId>
+            <version>4.5.13</version>
+        </dependency>
+
+        <!-- This is used to run the local Pinot -->
+        <dependency>
+            <groupId>com.github.docker-java</groupId>
+            <artifactId>docker-java</artifactId>
+            <version>3.2.7</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.github.docker-java</groupId>
+            <artifactId>docker-java-transport-httpclient5</artifactId>
+            <version>3.2.7</version>
+            <scope>test</scope>
+        </dependency>
+        <!-- This is used to interact with Pinot Broker -->
+        <dependency>
+            <groupId>org.apache.pinot</groupId>
+            <artifactId>pinot-java-client</artifactId>
+            <version>${pinot.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-test-utils_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-tests</artifactId>
+            <version>${flink.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-runtime_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-module-junit4</artifactId>
+            <version>1.5.5</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-api-mockito</artifactId>
+            <version>1.5.5</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>testcontainers</artifactId>
+            <version>${testcontainers.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>junit-jupiter</artifactId>
+            <version>${testcontainers.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-enforcer-plugin</artifactId>
+                <version>1.4.1</version>
+                <executions>
+                    <execution>
+                        <id>enforce-versions</id>
+                        <goals>
+                            <goal>enforce</goal>
+                        </goals>
+                        <configuration>
+                            <rules>
+                                <requireJavaVersion>
+                                    <version>${java.version}</version>
+                                </requireJavaVersion>
+                                <bannedDependencies>
+                                    <excludes combine.self="override">
+                                        <!-- The org.codehaus.groovy dependency is required by org.apache.pinot:pinot-tools -->
+                                    </excludes>
+                                    <searchTransitive>true</searchTransitive>
+                                </bannedDependencies>
+                            </rules>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <version>3.1.1</version>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <version>3.1.0</version>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <transformers>
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
+                                    <resource>reference.conf</resource>
+                                </transformer>
+                            </transformers>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>

Review comment:
       The [main pom.xml](https://github.com/apache/bahir-flink/blob/b618a77cca2e44be86f18607e21a6b2d2bf8dd39/pom.xml#L237) excludes the `org.codehaus.groovy` dependency, but `org.apache.pinot:pinot-tools` requires it, so we have to override the exclude.




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594295810



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();

Review comment:
       done




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594709294



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();

Review comment:
       I totally agree that it would have been much cleaner if the `SinkGlobalCommitter` was able to notify the `SinkWriter`. This would e.g. make the external filesystem obsolete.  
   But this dataflow is currently not supported by the Flink sink interface.




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

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



[GitHub] [bahir-flink] eskabetxe commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
eskabetxe commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r592227559



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.HttpEntity;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHost;
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        this.controllerHost = checkNotNull(controllerHost);

Review comment:
       this could be removed, its only used to generate the controolerHostPort no?

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,428 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+                e.printStackTrace();
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        public CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    static class SegmentCommitter implements Callable<Boolean> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class);
+
+        final String pinotControllerHost;
+        final String pinotControllerPort;
+        final String tempDirPrefix;
+        final FileSystemAdapter fsAdapter;
+        final String dataFilePath;
+        final String segmentName;
+        final Schema tableSchema;
+        final TableConfig tableConfig;
+        final String timeColumnName;
+        final TimeUnit segmentTimeUnit;
+
+        /**
+         * @param pinotControllerHost Host of the Pinot controller
+         * @param pinotControllerPort Port of the Pinot controller
+         * @param fsAdapter           Filesystem adapter used to load data files from the shared file system
+         * @param dataFilePath        Data file to load from the shared file system
+         * @param segmentName         Name of the segment to create and commit
+         * @param tableSchema         Pinot table schema
+         * @param tableConfig         Pinot table config
+         * @param timeColumnName      Name of the column containing the timestamp
+         * @param segmentTimeUnit     Unit of the time column
+         */
+        public SegmentCommitter(String pinotControllerHost, String pinotControllerPort, String tempDirPrefix, FileSystemAdapter fsAdapter, String dataFilePath, String segmentName, Schema tableSchema, TableConfig tableConfig, String timeColumnName, TimeUnit segmentTimeUnit) {
+            this.pinotControllerHost = pinotControllerHost;
+            this.pinotControllerPort = pinotControllerPort;
+            this.tempDirPrefix = tempDirPrefix;
+            this.fsAdapter = fsAdapter;
+            this.dataFilePath = dataFilePath;
+            this.segmentName = segmentName;
+            this.tableSchema = tableSchema;
+            this.tableConfig = tableConfig;
+            this.timeColumnName = timeColumnName;
+            this.segmentTimeUnit = segmentTimeUnit;
+        }
+
+        /**
+         * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment
+         * and finally uploads the segment to the Pinot controller
+         *
+         * @return True if the commit succeeded
+         */
+        @Override
+        public Boolean call() {
+            try {
+                // Download data file from the shared filesystem
+                LOG.info("Downloading data file {} from shared file system...", dataFilePath);
+                File segmentData = fsAdapter.copyToLocalFile(dataFilePath);
+                LOG.info("Successfully downloaded data file {} from shared file system", dataFilePath);
+
+                File segmentFile = Files.createTempDirectory(this.tempDirPrefix).toFile();
+                LOG.info("Creating segment in " + segmentFile.getAbsolutePath());
+
+                // Creates a segment with name `segmentName` in `segmentFile`
+                this.generateSegment(segmentData, segmentFile, true);
+
+                // Uploads the recently created segment to the Pinot controller
+                this.uploadSegment(segmentFile);
+
+                // Commit successful
+                return true;
+            } catch (IOException e) {
+                e.printStackTrace();
+                LOG.error(e.getMessage());
+
+                // Commit failed
+                return false;
+            }
+        }
+
+        /**
+         * Creates a segment from the given parameters.
+         * This method was adapted from {@link org.apache.pinot.tools.admin.command.CreateSegmentCommand}.
+         *
+         * @param dataFile                  File containing the JSON data
+         * @param outDir                    Segment target path
+         * @param _postCreationVerification Verify segment after generation
+         */
+        public void generateSegment(File dataFile, File outDir, Boolean _postCreationVerification) {
+            SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(tableConfig, tableSchema);
+            segmentGeneratorConfig.setSegmentName(segmentName);
+            segmentGeneratorConfig.setSegmentTimeUnit(segmentTimeUnit);
+            segmentGeneratorConfig.setTimeColumnName(timeColumnName);
+            segmentGeneratorConfig.setInputFilePath(dataFile.getPath());
+            segmentGeneratorConfig.setFormat(FileFormat.JSON);
+            segmentGeneratorConfig.setOutDir(outDir.getPath());
+            segmentGeneratorConfig.setTableName(tableConfig.getTableName());
+
+            try {
+                SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+                driver.init(segmentGeneratorConfig);
+                driver.build();
+                File indexDir = new File(outDir, segmentName);
+                LOG.info("Successfully created segment: {} in directory: {}", segmentName, indexDir);
+                if (_postCreationVerification) {
+                    LOG.info("Verifying the segment by loading it");
+                    ImmutableSegment segment = ImmutableSegmentLoader.load(indexDir, ReadMode.mmap);
+                    LOG.info("Successfully loaded segment: {} of size: {} bytes", segmentName,
+                            segment.getSegmentSizeBytes());
+                    segment.destroy();
+                }
+            } catch (Exception e) {
+                e.printStackTrace();

Review comment:
       log this, or remove

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.HttpEntity;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHost;
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        this.controllerHost = checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+
+            String body = EntityUtils.toString(response.getEntity());

Review comment:
       if fail to connect to pinot this will fail no?

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,428 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+                e.printStackTrace();
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        public CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    static class SegmentCommitter implements Callable<Boolean> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class);
+
+        final String pinotControllerHost;
+        final String pinotControllerPort;
+        final String tempDirPrefix;
+        final FileSystemAdapter fsAdapter;
+        final String dataFilePath;
+        final String segmentName;
+        final Schema tableSchema;
+        final TableConfig tableConfig;
+        final String timeColumnName;
+        final TimeUnit segmentTimeUnit;
+
+        /**
+         * @param pinotControllerHost Host of the Pinot controller
+         * @param pinotControllerPort Port of the Pinot controller
+         * @param fsAdapter           Filesystem adapter used to load data files from the shared file system
+         * @param dataFilePath        Data file to load from the shared file system
+         * @param segmentName         Name of the segment to create and commit
+         * @param tableSchema         Pinot table schema
+         * @param tableConfig         Pinot table config
+         * @param timeColumnName      Name of the column containing the timestamp
+         * @param segmentTimeUnit     Unit of the time column
+         */
+        public SegmentCommitter(String pinotControllerHost, String pinotControllerPort, String tempDirPrefix, FileSystemAdapter fsAdapter, String dataFilePath, String segmentName, Schema tableSchema, TableConfig tableConfig, String timeColumnName, TimeUnit segmentTimeUnit) {
+            this.pinotControllerHost = pinotControllerHost;
+            this.pinotControllerPort = pinotControllerPort;
+            this.tempDirPrefix = tempDirPrefix;
+            this.fsAdapter = fsAdapter;
+            this.dataFilePath = dataFilePath;
+            this.segmentName = segmentName;
+            this.tableSchema = tableSchema;
+            this.tableConfig = tableConfig;
+            this.timeColumnName = timeColumnName;
+            this.segmentTimeUnit = segmentTimeUnit;
+        }
+
+        /**
+         * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment
+         * and finally uploads the segment to the Pinot controller
+         *
+         * @return True if the commit succeeded
+         */
+        @Override
+        public Boolean call() {
+            try {
+                // Download data file from the shared filesystem
+                LOG.info("Downloading data file {} from shared file system...", dataFilePath);
+                File segmentData = fsAdapter.copyToLocalFile(dataFilePath);
+                LOG.info("Successfully downloaded data file {} from shared file system", dataFilePath);
+
+                File segmentFile = Files.createTempDirectory(this.tempDirPrefix).toFile();
+                LOG.info("Creating segment in " + segmentFile.getAbsolutePath());
+
+                // Creates a segment with name `segmentName` in `segmentFile`
+                this.generateSegment(segmentData, segmentFile, true);
+
+                // Uploads the recently created segment to the Pinot controller
+                this.uploadSegment(segmentFile);
+
+                // Commit successful
+                return true;
+            } catch (IOException e) {
+                e.printStackTrace();

Review comment:
       set this to log

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/emulator/PinotEmulatorManager.java
##########
@@ -0,0 +1,341 @@
+/*
+ * 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.flink.streaming.connectors.pinot.emulator;
+
+import com.github.dockerjava.api.DockerClient;

Review comment:
       could you use testcontainers?

##########
File path: flink-connector-pinot/pom.xml
##########
@@ -0,0 +1,213 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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/maven-v4_0_0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.bahir</groupId>
+        <artifactId>bahir-flink-parent_2.11</artifactId>
+        <version>1.1-SNAPSHOT</version>
+        <relativePath>..</relativePath>
+    </parent>
+
+    <artifactId>flink-connector-pinot_2.11</artifactId>
+    <name>flink-connector-pinot</name>
+
+
+    <packaging>jar</packaging>
+
+    <!-- Allow users to pass custom connector versions -->
+    <properties>
+        <pinot.version>0.6.0</pinot.version>
+
+        <!-- Flink version -->
+        <flink.version>1.12.0</flink.version>

Review comment:
       we should update the flink version on parent
   

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotWriterSegment.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@link PinotWriterSegment} represents exactly one segment that can be found in the Pinot
+ * cluster once the commit has been completed.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotWriterSegment<IN> implements Serializable {
+
+    private static final Logger LOG = LoggerFactory.getLogger("PinotWriterSegment");
+
+    private final int maxRowsPerSegment;
+    private final String tempDirPrefix;
+    private final JsonSerializer<IN> jsonSerializer;
+    private final FileSystemAdapter fsAdapter;
+
+    private boolean acceptsElements = true;
+
+    private final List<IN> elements;
+    private File dataFile;
+    private long minTimestamp = Long.MAX_VALUE;
+    private long maxTimestamp = Long.MIN_VALUE;
+
+    /**
+     * @param maxRowsPerSegment Maximum number of rows to be stored within a Pinot segment
+     * @param tempDirPrefix     Prefix for temp directories used
+     * @param jsonSerializer    Serializer used to convert elements to JSON
+     * @param fsAdapter         Filesystem adapter used to save files for sharing files across nodes
+     */
+    protected PinotWriterSegment(int maxRowsPerSegment, String tempDirPrefix, JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        checkArgument(maxRowsPerSegment > 0L);
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.elements = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements and stores them in memory until either {@link #maxRowsPerSegment} is reached
+     * or {@link #prepareCommit} is called.
+     *
+     * @param element   Object from upstream task
+     * @param timestamp Timestamp assigned to element
+     * @throws IOException
+     */
+    public void write(IN element, long timestamp) throws IOException {
+        if (!this.acceptsElements()) {
+            throw new IllegalStateException("This PinotSegmentWriter does not accept any elements anymore.");
+        }
+        this.elements.add(element);
+        this.minTimestamp = Long.min(this.minTimestamp, timestamp);
+        this.maxTimestamp = Long.max(this.maxTimestamp, timestamp);
+
+        // Writes elements to local filesystem once the maximum number of items is reached
+        if (this.elements.size() == this.maxRowsPerSegment) {

Review comment:
       why we write to local?




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

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



[GitHub] [bahir-flink] fapaul commented on pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
fapaul commented on pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#issuecomment-805874586


   @eskabetxe @rmetzger can some of you take another look, please? :)


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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594699621



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        public CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    static class SegmentCommitter implements Callable<Boolean> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class);
+
+        final String pinotControllerHost;
+        final String pinotControllerPort;
+        final String tempDirPrefix;
+        final FileSystemAdapter fsAdapter;
+        final String dataFilePath;
+        final String segmentName;
+        final Schema tableSchema;
+        final TableConfig tableConfig;
+        final String timeColumnName;
+        final TimeUnit segmentTimeUnit;
+
+        /**
+         * @param pinotControllerHost Host of the Pinot controller
+         * @param pinotControllerPort Port of the Pinot controller
+         * @param fsAdapter           Filesystem adapter used to load data files from the shared file system
+         * @param dataFilePath        Data file to load from the shared file system
+         * @param segmentName         Name of the segment to create and commit
+         * @param tableSchema         Pinot table schema
+         * @param tableConfig         Pinot table config
+         * @param timeColumnName      Name of the column containing the timestamp
+         * @param segmentTimeUnit     Unit of the time column
+         */
+        public SegmentCommitter(String pinotControllerHost, String pinotControllerPort, String tempDirPrefix, FileSystemAdapter fsAdapter, String dataFilePath, String segmentName, Schema tableSchema, TableConfig tableConfig, String timeColumnName, TimeUnit segmentTimeUnit) {
+            this.pinotControllerHost = pinotControllerHost;
+            this.pinotControllerPort = pinotControllerPort;
+            this.tempDirPrefix = tempDirPrefix;
+            this.fsAdapter = fsAdapter;
+            this.dataFilePath = dataFilePath;
+            this.segmentName = segmentName;
+            this.tableSchema = tableSchema;
+            this.tableConfig = tableConfig;
+            this.timeColumnName = timeColumnName;
+            this.segmentTimeUnit = segmentTimeUnit;
+        }
+
+        /**
+         * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment
+         * and finally uploads the segment to the Pinot controller
+         *
+         * @return True if the commit succeeded
+         */
+        @Override
+        public Boolean call() {
+            try {
+                // Download data file from the shared filesystem
+                LOG.info("Downloading data file {} from shared file system...", dataFilePath);
+                File segmentData = fsAdapter.copyToLocalFile(dataFilePath);
+                LOG.info("Successfully downloaded data file {} from shared file system", dataFilePath);
+
+                File segmentFile = Files.createTempDirectory(this.tempDirPrefix).toFile();

Review comment:
       The global committer now creates one temp directory which is deleted up onClose. The temp files created during segment commit are deleted after successful or failed commits.




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594684898



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);

Review comment:
       The idea of returning the failed commits here, is that the "commit-with-retry" pattern can be used once implemented. I would suppose that the number of retries will be definable by the user to prevent endless retrying commits.




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r596174677



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerHttpClient.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerHttpClient implements Closeable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerHttpClient.class);
+    protected final String controllerHostPort;
+    protected final CloseableHttpClient httpClient;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerHttpClient(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+        httpClient = HttpClients.createDefault();
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpResponse response = httpClient.execute(request)) {
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.debug("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {
+        HttpGet httpget = new HttpGet(this.controllerHostPort + path);
+        LOG.debug("Sending GET request to {}", path);
+        return this.execute(httpget);
+    }
+
+    /**
+     * Issues a DELETE request to the Pinot controller API.
+     *
+     * @param path Path to issue DELETE request to
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse delete(String path) throws IOException {

Review comment:
       We need to expose this externally, used e.g. in `PinotControllerClient`

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerHttpClient.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerHttpClient implements Closeable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerHttpClient.class);
+    protected final String controllerHostPort;
+    protected final CloseableHttpClient httpClient;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerHttpClient(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+        httpClient = HttpClients.createDefault();
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpResponse response = httpClient.execute(request)) {
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {

Review comment:
       We need to expose this externally, used e.g. in `PinotControllerClient`

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerHttpClient.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerHttpClient implements Closeable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerHttpClient.class);
+    protected final String controllerHostPort;
+    protected final CloseableHttpClient httpClient;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerHttpClient(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+        httpClient = HttpClients.createDefault();
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpResponse response = httpClient.execute(request)) {
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.debug("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {

Review comment:
       We need to expose this externally, used e.g. in `PinotControllerClient`




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594726779



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.HttpEntity;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHost;
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        this.controllerHost = checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+
+            String body = EntityUtils.toString(response.getEntity());

Review comment:
       `httpClient.execute(request)` will raise an `HttpHostConnectException`(`IOException`), if the connection could not be established.




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

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



[GitHub] [bahir-flink] fapaul commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
fapaul commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r598197430



##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotSinkTest.java
##########
@@ -0,0 +1,396 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.connector.sink.SinkWriter;
+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.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.flink.util.Preconditions;
+import org.apache.pinot.client.PinotClientException;
+import org.apache.pinot.client.ResultSet;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.opentest4j.AssertionFailedError;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * E2e tests for Pinot Sink using BATCH and STREAMING execution mode
+ */
+public class PinotSinkTest extends PinotTestBase {
+
+    private static final int MAX_ROWS_PER_SEGMENT = 5;
+    private static final long STREAMING_CHECKPOINTING_INTERVAL = 50;
+    private static final int DATA_CHECKING_TIMEOUT_SECONDS = 60;
+    private static final AtomicBoolean hasFailedOnce = new AtomicBoolean(false);
+    private static CountDownLatch latch;
+
+    @BeforeEach
+    public void setUp() throws IOException {
+        super.setUp();
+        // Reset hasFailedOnce flag used during failure recovery testing before each test.
+        hasFailedOnce.set(false);
+        // Reset latch used to keep the generator streaming source up until the test is completed.
+        latch = new CountDownLatch(1);
+    }
+
+    /**
+     * Tests the BATCH execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testBatchSink() throws Exception {
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.BATCH);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+        env.setParallelism(2);
+
+        List<String> rawData = getRawTestData(12);
+        DataStream<SingleColumnTableRow> dataStream = setupBatchDataSource(env, rawData);
+        setupSink(dataStream);
+
+        // Run
+        env.execute();
+
+        // Check for data in Pinot
+        checkForDataInPinotWithRetry(rawData);
+    }
+
+    /**
+     * Tests failure recovery of the {@link PinotSink} using BATCH execution mode.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testFailureRecoveryInBatchingSink() throws Exception {
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.BATCH);
+        env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 10));
+        env.setParallelism(2);
+
+        List<String> rawData = getRawTestData(12);
+        DataStream<SingleColumnTableRow> dataStream = setupBatchDataSource(env, rawData);
+        dataStream = setupFailingMapper(dataStream, 8);
+        setupSink(dataStream);
+
+        // Run
+        env.execute();
+
+        // Check for data in Pinot
+        checkForDataInPinotWithRetry(rawData);
+    }
+
+    /**
+     * Tests the STREAMING execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testStreamingSink() throws Exception {
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.STREAMING);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+        env.setParallelism(2);
+        env.enableCheckpointing(STREAMING_CHECKPOINTING_INTERVAL);
+
+        List<String> rawData = getRawTestData(20);
+        DataStream<SingleColumnTableRow> dataStream = setupStreamingDataSource(env, rawData);
+        setupSink(dataStream);
+
+        // Start execution of job
+        env.executeAsync();
+
+        // Check for data in Pinot
+        checkForDataInPinotWithRetry(rawData);
+
+        // Generator source can now shut down
+        latch.countDown();
+    }
+
+    /**
+     * Tests failure recovery of the {@link PinotSink} using STREAMING execution mode.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testFailureRecoveryInStreamingSink() throws Exception {
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.STREAMING);
+        env.setParallelism(1);
+        env.enableCheckpointing(STREAMING_CHECKPOINTING_INTERVAL);
+
+        List<String> rawData = getRawTestData(20);
+        DataStream<SingleColumnTableRow> dataStream = setupStreamingDataSource(env, rawData);
+        // With a segment size of MAX_ROWS_PER_SEGMENT = 5 elements and a parallelism of 2,
+        // the failure will be raised once the first 2 segments were committed to Pinot.
+        dataStream = setupFailingMapper(dataStream, 12);
+        setupSink(dataStream);
+
+        // Start execution of job
+        env.executeAsync();
+
+        // Check for data in Pinot
+        checkForDataInPinotWithRetry(rawData);
+
+        // Generator source can now shut down
+        latch.countDown();
+    }
+
+    /**
+     * Generates a small test dataset consisting of {@link SingleColumnTableRow}s.
+     *
+     * @return List of SingleColumnTableRow
+     */
+    private List<String> getRawTestData(int numItems) {
+        return IntStream.range(1, numItems + 1)
+                .mapToObj(num -> "ColValue" + num)
+                .collect(Collectors.toList());
+    }
+
+    /**
+     * Setup the data source for STREAMING tests.
+     *
+     * @param env           Stream execution environment
+     * @param rawDataValues Data values to send
+     * @return resulting data stream
+     */
+    private DataStream<SingleColumnTableRow> setupStreamingDataSource(StreamExecutionEnvironment env, List<String> rawDataValues) {
+        SimpleStreamingSource source = new SimpleStreamingSource(rawDataValues, 10);
+        return env.addSource(source)
+                .name("Test input");
+    }
+
+    /**
+     * Setup the data source for BATCH tests.
+     *
+     * @param env           Stream execution environment
+     * @param rawDataValues Data values to send
+     * @return resulting data stream
+     */
+    private DataStream<SingleColumnTableRow> setupBatchDataSource(StreamExecutionEnvironment env, List<String> rawDataValues) {
+        return env.fromCollection(rawDataValues)
+                .map(value -> new SingleColumnTableRow(value, System.currentTimeMillis()))
+                .name("Test input");
+    }
+
+    /**
+     * Setup a mapper that fails when processing the nth element with n = failOnceAtNthElement.
+     *
+     * @param dataStream           Input data stream
+     * @param failOnceAtNthElement Number of elements to process before raising the exception
+     * @return resulting data stream
+     */
+    private DataStream<SingleColumnTableRow> setupFailingMapper(DataStream<SingleColumnTableRow> dataStream, int failOnceAtNthElement) {
+        AtomicInteger messageCounter = new AtomicInteger(0);
+
+        return dataStream.map(element -> {
+            if (!hasFailedOnce.get() && messageCounter.incrementAndGet() == failOnceAtNthElement) {
+                hasFailedOnce.set(true);
+                // Wait more than STREAMING_CHECKPOINTING_INTERVAL to ensure
+                // that at least one checkpoint was created before raising the exception.
+                Thread.sleep(4 * STREAMING_CHECKPOINTING_INTERVAL);

Review comment:
       Maybe you can find a way to also remove this timeout :) One idea would be to make your map function implement `CheckpointedFunction` which allows you to receive the checkpoint notification.
   
   I also do not really know why it is important for the failure to have at-least one checkpoint.




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r596184664



##########
File path: flink-connector-pinot/README.md
##########
@@ -0,0 +1,126 @@
+# Flink Pinot Connector
+
+This connector provides a sink to [Apache Pinot](http://pinot.apache.org/)™.  
+To use this connector, add the following dependency to your project:
+
+    <dependency>
+      <groupId>org.apache.bahir</groupId>
+      <artifactId>flink-connector-pinot_2.11</artifactId>
+      <version>1.1-SNAPSHOT</version>
+    </dependency>
+
+*Version Compatibility*: This module is compatible with Pinot 0.6.0.
+
+Note that the streaming connectors are not part of the binary distribution of Flink. You need to link them into your job jar for cluster execution.
+See how to link with them for cluster execution [here](https://ci.apache.org/projects/flink/flink-docs-release-1.2/dev/linking.html).
+
+The sink class is called `PinotSink`.
+
+## Usage
+```java
+StreamExecutionEnvironment env = ...
+// Checkpointing needs to be enabled when executing in STREAMING mode
+        env.enableCheckpointing(long interval);
+
+        DataStream<PinotRow> dataStream = ...
+        PinotSink pinotSink = new PinotSink.Builder<PinotRow>(String pinotControllerHost, String pinotControllerPort, String tableName)
+
+        // Serializes a PinotRow to JSON format
+        .withJsonSerializer(JsonSerializer<PinotRow> jsonSerializer)
+
+        // Extracts the timestamp from a PinotRow
+        .withEventTimeExtractor(EventTimeExtractor<IN> eventTimeExtractor)
+
+        // Defines the segment name generation via the predefined SimpleSegmentNameGenerator
+        // Exemplary segment name: tableName_minTimestamp_maxTimestamp_segmentNamePostfix_0
+        .withSimpleSegmentNameGenerator(String tableName, String segmentNamePostfix)
+
+        // Use a custom segment name generator if the SimpleSegmentNameGenerator does not work for your use case
+        .withSegmentNameGenerator(SegmentNameGenerator segmentNameGenerator)
+
+        // Use the local filesystem to share committables across subTasks
+        // CAUTION: Use only if all subTasks run on the same node with access to the local filesystem
+        .withLocalFileSystemAdapter()
+
+        // Use a custom filesystem adapter. 
+        // CAUTION: Make sure all nodes your Flink app runs on can access the shared filesystem via the provided FileSystemAdapter
+        .withFileSystemAdapter(FileSystemAdapter fsAdapter)
+
+        // Defines the size of the Pinot segments
+        .withMaxRowsPerSegment(int maxRowsPerSegment)
+
+        // Prefix within the local filesystem's temp directory used for storing intermediate files
+        .withTempDirectoryPrefix(String tempDirPrefix)
+        
+        // Number of threads used in the `PinotSinkGlobalCommitter` to commit a batch of segments
+        // Optional - Default is 4
+        .withNumCommitThreads(int numCommitThreads)
+
+        // Builds the PinotSink
+        .build()
+        dataStream.addSink(pinotSink);
+```
+
+## Options
+| Option                 | Description                                                                      |
+| ---------------------- | -------------------------------------------------------------------------------- | 
+| `pinotControllerHost`  | Host of the Pinot controller                                                     |
+| `pinotControllerPort`  | Port of the Pinot controller                                                     |
+| `tableName`            | Target Pinot table's name                                                        |
+| `maxRowsPerSegment`    | Maximum number of rows to be stored within a Pinot segment                       |
+| `tempDirPrefix`         | Prefix for temp directories used                                                  |
+| `jsonSerializer`       | Serializer used to convert elements to JSON                                      |
+| `eventTimeExtractor`   | Defines the way event times are extracted from received objects                   |
+| `segmentNameGenerator` | Pinot segment name generator                                                     |
+| `fsAdapter`            | Filesystem adapter used to save files for sharing files across nodes               |
+| `numCommitThreads`     | Number of threads used in the `PinotSinkGlobalCommitter` for committing segments |
+
+## Architecture

Review comment:
       Totally 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.

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r597987741



##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotSinkTest.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.pinot.client.ResultSet;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.opentest4j.AssertionFailedError;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * E2e tests for Pinot Sink using BATCH and STREAMING execution mode
+ */
+public class PinotSinkTest extends PinotTestBase {
+
+    /**
+     * Tests the BATCH execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testBatchSink() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.BATCH);
+        env.setParallelism(2);
+
+        List<SingleColumnTableRow> data = getTestData(12);
+        this.setupDataStream(env, data);
+
+        // Run
+        executeOnMiniCluster(env.getStreamGraph().getJobGraph());
+
+        checkForDataInPinotWithRetry(data, 20);
+    }
+
+    /**
+     * Tests the STREAMING execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testStreamingSink() throws Exception {
+        final Configuration conf = new Configuration();
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(conf);
+        env.setRuntimeMode(RuntimeExecutionMode.STREAMING);
+        env.setParallelism(2);
+        env.enableCheckpointing(50);
+
+        List<SingleColumnTableRow> data = getTestData(12);
+        this.setupDataStream(env, data);
+
+        // Run
+        executeOnMiniCluster(env.getStreamGraph().getJobGraph());
+
+        // We only expect the first 100 elements to be already committed to Pinot.
+        // The remaining would follow once we increase the input data size.
+        // The stream executions stops once the last input tuple was sent to the sink.
+        checkForDataInPinotWithRetry(data, 20);
+    }
+
+    /**
+     * Generates a small test dataset consisting of {@link SingleColumnTableRow}s.
+     *
+     * @return List of SingleColumnTableRow
+     */
+    private List<SingleColumnTableRow> getTestData(int numItems) {
+        return IntStream.range(1, numItems + 1)
+                .mapToObj(num -> "ColValue" + num)
+                .map(col1 -> new SingleColumnTableRow(col1, System.currentTimeMillis()))
+                .collect(Collectors.toList());
+    }
+
+    /**
+     * Executes a given JobGraph on a MiniCluster.
+     *
+     * @param jobGraph JobGraph to execute
+     * @throws Exception
+     */
+    private void executeOnMiniCluster(JobGraph jobGraph) throws Exception {
+        final Configuration config = new Configuration();
+        config.setString(RestOptions.BIND_PORT, "18081-19000");
+        final MiniClusterConfiguration cfg =
+                new MiniClusterConfiguration.Builder()
+                        .setNumTaskManagers(1)
+                        .setNumSlotsPerTaskManager(4)
+                        .setConfiguration(config)
+                        .build();
+
+        try (MiniCluster miniCluster = new MiniCluster(cfg)) {
+            miniCluster.start();
+            miniCluster.executeJobBlocking(jobGraph);
+        }
+    }
+
+    /**
+     * Sets up a DataStream using the provided execution environment and the provided input data.
+     *
+     * @param env  stream execution environment
+     * @param data Input data
+     */
+    private void setupDataStream(StreamExecutionEnvironment env, List<SingleColumnTableRow> data) {
+        // Create test stream
+        DataStream<SingleColumnTableRow> theData =
+                env.fromCollection(data)
+                        .name("Test input");
+
+        String tempDirPrefix = "flink-pinot-connector-test";
+        PinotSinkSegmentNameGenerator segmentNameGenerator = new SimpleSegmentNameGenerator(TABLE_NAME, "flink-connector");
+        FileSystemAdapter fsAdapter = new LocalFileSystemAdapter(tempDirPrefix);
+        JsonSerializer<SingleColumnTableRow> jsonSerializer = new SingleColumnTableRowSerializer();
+
+        EventTimeExtractor<SingleColumnTableRow> eventTimeExtractor = new SingleColumnTableRowEventTimeExtractor();
+
+        PinotSink<SingleColumnTableRow> sink = new PinotSink.Builder<SingleColumnTableRow>(getPinotHost(), getPinotControllerPort(), TABLE_NAME)
+                .withMaxRowsPerSegment(5)
+                .withTempDirectoryPrefix(tempDirPrefix)
+                .withJsonSerializer(jsonSerializer)
+                .withEventTimeExtractor(eventTimeExtractor)
+                .withSegmentNameGenerator(segmentNameGenerator)
+                .withFileSystemAdapter(fsAdapter)
+                .withNumCommitThreads(2)
+                .build();
+
+        // Sink into Pinot
+        theData.sinkTo(sink).name("Pinot sink");
+    }
+
+    /**
+     * As Pinot might take some time to index the recently pushed segments we might need to retry
+     * the {@link #checkForDataInPinot} method multiple times. This method provides a simple wrapper
+     * using linear retry backoff delay.
+     *
+     * @param data                  Data to expect in the Pinot table
+     * @param retryTimeoutInSeconds Maximum duration in seconds to wait for the data to arrive
+     * @throws InterruptedException
+     */
+    private void checkForDataInPinotWithRetry(List<SingleColumnTableRow> data, int retryTimeoutInSeconds) throws InterruptedException {
+        long endTime = System.currentTimeMillis() + 1000L * retryTimeoutInSeconds;
+        // Use max 10 retries with linear retry backoff delay
+        long retryDelay = 1000L / 10 * retryTimeoutInSeconds;
+        do {
+            try {
+                checkForDataInPinot(data);
+                // In case of no error, we can skip further retries
+                return;
+            } catch (AssertionFailedError | PinotControllerApiException e) {

Review comment:
       It could still be that Pinot just hasn't (fully) finished indexing all the pushed segments when we check for the data.
   Nonetheless, I spotted an error here: It could be that we catch the `AssertionFailedError`, sleep for some time and then finally do not execute the check once again as the `retryTimeoutInSeconds` was exceeded. So although the test actually should have failed (data not in Pinot), we would not raise an exception.
   
   In order to prevent this, I've added a final check whether all data is in Pinot.




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

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



[GitHub] [bahir-flink] fapaul commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
fapaul commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r595813602



##########
File path: flink-connector-pinot/pom.xml
##########
@@ -0,0 +1,225 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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/maven-v4_0_0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.bahir</groupId>
+        <artifactId>bahir-flink-parent_2.11</artifactId>
+        <version>1.1-SNAPSHOT</version>
+        <relativePath>..</relativePath>
+    </parent>
+
+    <artifactId>flink-connector-pinot_2.11</artifactId>
+    <name>flink-connector-pinot</name>
+
+
+    <packaging>jar</packaging>
+
+    <!-- Allow users to pass custom connector versions -->
+    <properties>
+        <pinot.version>0.6.0</pinot.version>
+
+        <testcontainers.version>1.15.2</testcontainers.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-scala_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+        </dependency>
+
+        <!-- This is used to interact with Pinot Controller -->
+        <dependency>
+            <groupId>org.apache.pinot</groupId>
+            <artifactId>pinot-tools</artifactId>
+            <version>${pinot.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpmime</artifactId>
+            <version>4.5.13</version>
+        </dependency>
+
+        <!-- This is used to run the local Pinot -->
+        <dependency>
+            <groupId>com.github.docker-java</groupId>
+            <artifactId>docker-java</artifactId>
+            <version>3.2.7</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.github.docker-java</groupId>
+            <artifactId>docker-java-transport-httpclient5</artifactId>
+            <version>3.2.7</version>
+            <scope>test</scope>
+        </dependency>
+        <!-- This is used to interact with Pinot Broker -->
+        <dependency>
+            <groupId>org.apache.pinot</groupId>
+            <artifactId>pinot-java-client</artifactId>
+            <version>${pinot.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-test-utils_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-tests</artifactId>
+            <version>${flink.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-runtime_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-module-junit4</artifactId>
+            <version>1.5.5</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-api-mockito</artifactId>
+            <version>1.5.5</version>
+            <scope>test</scope>

Review comment:
       Are these dependencies used?

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/segment/name/PinotSinkSegmentNameGenerator.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.flink.streaming.connectors.pinot.segment.name;
+
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+
+import javax.annotation.Nullable;
+import java.io.Serializable;
+
+/**
+ * Defines the segment name generator interface that is used to generate segment names.
+ */
+public abstract class PinotSinkSegmentNameGenerator implements SegmentNameGenerator, Serializable {

Review comment:
       I guess this can be an interface as well?

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerHttpClient.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerHttpClient implements Closeable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerHttpClient.class);
+    protected final String controllerHostPort;
+    protected final CloseableHttpClient httpClient;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerHttpClient(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+        httpClient = HttpClients.createDefault();
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpResponse response = httpClient.execute(request)) {
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.debug("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {
+        HttpGet httpget = new HttpGet(this.controllerHostPort + path);
+        LOG.debug("Sending GET request to {}", path);
+        return this.execute(httpget);
+    }
+
+    /**
+     * Issues a DELETE request to the Pinot controller API.
+     *
+     * @param path Path to issue DELETE request to
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse delete(String path) throws IOException {

Review comment:
       private

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerHttpClient.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerHttpClient implements Closeable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerHttpClient.class);
+    protected final String controllerHostPort;
+    protected final CloseableHttpClient httpClient;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerHttpClient(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+        httpClient = HttpClients.createDefault();
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpResponse response = httpClient.execute(request)) {
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {

Review comment:
       private

##########
File path: flink-connector-pinot/pom.xml
##########
@@ -0,0 +1,225 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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/maven-v4_0_0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.bahir</groupId>
+        <artifactId>bahir-flink-parent_2.11</artifactId>
+        <version>1.1-SNAPSHOT</version>
+        <relativePath>..</relativePath>
+    </parent>
+
+    <artifactId>flink-connector-pinot_2.11</artifactId>
+    <name>flink-connector-pinot</name>
+
+
+    <packaging>jar</packaging>
+
+    <!-- Allow users to pass custom connector versions -->
+    <properties>
+        <pinot.version>0.6.0</pinot.version>
+
+        <testcontainers.version>1.15.2</testcontainers.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-scala_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+        </dependency>
+
+        <!-- This is used to interact with Pinot Controller -->
+        <dependency>
+            <groupId>org.apache.pinot</groupId>
+            <artifactId>pinot-tools</artifactId>
+            <version>${pinot.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpmime</artifactId>
+            <version>4.5.13</version>
+        </dependency>
+
+        <!-- This is used to run the local Pinot -->
+        <dependency>
+            <groupId>com.github.docker-java</groupId>
+            <artifactId>docker-java</artifactId>
+            <version>3.2.7</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.github.docker-java</groupId>
+            <artifactId>docker-java-transport-httpclient5</artifactId>
+            <version>3.2.7</version>
+            <scope>test</scope>
+        </dependency>
+        <!-- This is used to interact with Pinot Broker -->
+        <dependency>
+            <groupId>org.apache.pinot</groupId>
+            <artifactId>pinot-java-client</artifactId>
+            <version>${pinot.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-test-utils_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-tests</artifactId>
+            <version>${flink.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-runtime_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-module-junit4</artifactId>
+            <version>1.5.5</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-api-mockito</artifactId>
+            <version>1.5.5</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>testcontainers</artifactId>
+            <version>${testcontainers.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>junit-jupiter</artifactId>
+            <version>${testcontainers.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-enforcer-plugin</artifactId>
+                <version>1.4.1</version>
+                <executions>
+                    <execution>
+                        <id>enforce-versions</id>
+                        <goals>
+                            <goal>enforce</goal>
+                        </goals>
+                        <configuration>
+                            <rules>
+                                <requireJavaVersion>
+                                    <version>${java.version}</version>
+                                </requireJavaVersion>
+                                <bannedDependencies>
+                                    <excludes combine.self="override">
+                                        <!-- The org.codehaus.groovy dependency is required by org.apache.pinot:pinot-tools -->
+                                    </excludes>
+                                    <searchTransitive>true</searchTransitive>
+                                </bannedDependencies>
+                            </rules>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <version>3.1.1</version>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <version>3.1.0</version>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <transformers>
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
+                                    <resource>reference.conf</resource>
+                                </transformer>
+                            </transformers>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>

Review comment:
       Why do we need this?




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594301043



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.info("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {
+        HttpGet httpget = new HttpGet(this.controllerHostPort + path);
+        LOG.info("Sending GET request to {}", path);

Review comment:
       now using debug instead of info all over the place

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.info("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {
+        HttpGet httpget = new HttpGet(this.controllerHostPort + path);
+        LOG.info("Sending GET request to {}", path);
+        return this.execute(httpget);
+    }
+
+    /**
+     * Issues a DELETE request to the Pinot controller API.
+     *
+     * @param path Path to issue DELETE request to
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse delete(String path) throws IOException {
+        HttpDelete httpdelete = new HttpDelete(this.controllerHostPort + path);
+        LOG.info("Sending DELETE request to {}", path);

Review comment:
       now using debug instead of info all over the place

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.info("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {
+        HttpGet httpget = new HttpGet(this.controllerHostPort + path);
+        LOG.info("Sending GET request to {}", path);
+        return this.execute(httpget);
+    }
+
+    /**
+     * Issues a DELETE request to the Pinot controller API.
+     *
+     * @param path Path to issue DELETE request to
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse delete(String path) throws IOException {
+        HttpDelete httpdelete = new HttpDelete(this.controllerHostPort + path);
+        LOG.info("Sending DELETE request to {}", path);
+        return this.execute(httpdelete);
+    }
+
+    /**
+     * Checks whether the provided segment name is registered with the given table.
+     *
+     * @param tableName   Target table's name
+     * @param segmentName Segment name to check
+     * @return True if segment with the provided name exists
+     * @throws IOException
+     */
+    public boolean tableHasSegment(String tableName, String segmentName) throws IOException {
+        ApiResponse res = this.get(String.format("/tables/%s/%s/metadata", tableName, segmentName));
+
+        if (res.statusLine.getStatusCode() == 200) {
+            // A segment named `segmentName` exists within the table named `tableName`
+            return true;
+        }
+        if (res.statusLine.getStatusCode() == 404) {
+            // There is no such segment named `segmentName` within the table named `tableName`
+            // (or the table named `tableName` does not exist)
+            return false;
+        }
+
+        // Received an unexpected status code
+        throw new PinotControllerApiException(res.responseBody);
+    }
+
+    /**
+     * Deletes a segment from a table.
+     *
+     * @param tableName   Target table's name
+     * @param segmentName Identifies the segment to delete
+     * @throws IOException
+     */
+    public void deleteSegment(String tableName, String segmentName) throws IOException {
+        ApiResponse res = this.delete(String.format("/tables/%s/%s", tableName, segmentName));
+
+        if (res.statusLine.getStatusCode() != 200) {
+            LOG.error("Could not delete segment {} from table {}. Pinot controller returned: {}", tableName, segmentName, res.responseBody);
+            throw new PinotControllerApiException(res.responseBody);
+        }
+    }
+
+    /**
+     * Fetches a Pinot table's schema via the Pinot controller API.
+     *
+     * @param tableName Target table's name
+     * @return Pinot table schema
+     * @throws IOException
+     */
+    public Schema getSchema(String tableName) throws IOException {
+        Schema schema;
+        ApiResponse res = this.get(String.format("/tables/%s/schema", tableName));
+        LOG.info("Get schema request for table {} returned {}", tableName, res.responseBody);
+
+        if (res.statusLine.getStatusCode() != 200) {
+            throw new PinotControllerApiException(res.responseBody);
+        }
+
+        try {
+            schema = JsonUtils.stringToObject(res.responseBody, Schema.class);
+        } catch (Exception e) {
+            throw new IllegalStateException("Caught exception while reading schema from Pinot Controller's response: " + res.responseBody, e);
+        }
+        LOG.info("Retrieved schema: {}", schema.toSingleLineJsonString());

Review comment:
       now using debug instead of info all over the place

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.info("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {
+        HttpGet httpget = new HttpGet(this.controllerHostPort + path);
+        LOG.info("Sending GET request to {}", path);
+        return this.execute(httpget);
+    }
+
+    /**
+     * Issues a DELETE request to the Pinot controller API.
+     *
+     * @param path Path to issue DELETE request to
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse delete(String path) throws IOException {
+        HttpDelete httpdelete = new HttpDelete(this.controllerHostPort + path);
+        LOG.info("Sending DELETE request to {}", path);
+        return this.execute(httpdelete);
+    }
+
+    /**
+     * Checks whether the provided segment name is registered with the given table.
+     *
+     * @param tableName   Target table's name
+     * @param segmentName Segment name to check
+     * @return True if segment with the provided name exists
+     * @throws IOException
+     */
+    public boolean tableHasSegment(String tableName, String segmentName) throws IOException {
+        ApiResponse res = this.get(String.format("/tables/%s/%s/metadata", tableName, segmentName));
+
+        if (res.statusLine.getStatusCode() == 200) {
+            // A segment named `segmentName` exists within the table named `tableName`
+            return true;
+        }
+        if (res.statusLine.getStatusCode() == 404) {
+            // There is no such segment named `segmentName` within the table named `tableName`
+            // (or the table named `tableName` does not exist)
+            return false;
+        }
+
+        // Received an unexpected status code
+        throw new PinotControllerApiException(res.responseBody);
+    }
+
+    /**
+     * Deletes a segment from a table.
+     *
+     * @param tableName   Target table's name
+     * @param segmentName Identifies the segment to delete
+     * @throws IOException
+     */
+    public void deleteSegment(String tableName, String segmentName) throws IOException {
+        ApiResponse res = this.delete(String.format("/tables/%s/%s", tableName, segmentName));
+
+        if (res.statusLine.getStatusCode() != 200) {
+            LOG.error("Could not delete segment {} from table {}. Pinot controller returned: {}", tableName, segmentName, res.responseBody);
+            throw new PinotControllerApiException(res.responseBody);
+        }
+    }
+
+    /**
+     * Fetches a Pinot table's schema via the Pinot controller API.
+     *
+     * @param tableName Target table's name
+     * @return Pinot table schema
+     * @throws IOException
+     */
+    public Schema getSchema(String tableName) throws IOException {
+        Schema schema;
+        ApiResponse res = this.get(String.format("/tables/%s/schema", tableName));
+        LOG.info("Get schema request for table {} returned {}", tableName, res.responseBody);
+
+        if (res.statusLine.getStatusCode() != 200) {
+            throw new PinotControllerApiException(res.responseBody);
+        }
+
+        try {
+            schema = JsonUtils.stringToObject(res.responseBody, Schema.class);
+        } catch (Exception e) {
+            throw new IllegalStateException("Caught exception while reading schema from Pinot Controller's response: " + res.responseBody, e);
+        }
+        LOG.info("Retrieved schema: {}", schema.toSingleLineJsonString());
+        return schema;
+    }
+
+    /**
+     * Fetches a Pinot table's configuration via the Pinot controller API.
+     *
+     * @param tableName Target table's name
+     * @return Pinot table configuration
+     * @throws IOException
+     */
+    public TableConfig getTableConfig(String tableName) throws IOException {
+        TableConfig tableConfig;
+        ApiResponse res = this.get(String.format("/tables/%s", tableName));
+        LOG.info("Get table config request for table {} returned {}", tableName, res.responseBody);

Review comment:
       now using debug instead of info all over the place

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerApi.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerApi {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerApi.class);
+    protected final String controllerHostPort;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerApi(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpClient httpClient = HttpClients.createDefault();
+             CloseableHttpResponse response = httpClient.execute(request)) {
+
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.info("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {
+        HttpGet httpget = new HttpGet(this.controllerHostPort + path);
+        LOG.info("Sending GET request to {}", path);
+        return this.execute(httpget);
+    }
+
+    /**
+     * Issues a DELETE request to the Pinot controller API.
+     *
+     * @param path Path to issue DELETE request to
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse delete(String path) throws IOException {
+        HttpDelete httpdelete = new HttpDelete(this.controllerHostPort + path);
+        LOG.info("Sending DELETE request to {}", path);
+        return this.execute(httpdelete);
+    }
+
+    /**
+     * Checks whether the provided segment name is registered with the given table.
+     *
+     * @param tableName   Target table's name
+     * @param segmentName Segment name to check
+     * @return True if segment with the provided name exists
+     * @throws IOException
+     */
+    public boolean tableHasSegment(String tableName, String segmentName) throws IOException {
+        ApiResponse res = this.get(String.format("/tables/%s/%s/metadata", tableName, segmentName));
+
+        if (res.statusLine.getStatusCode() == 200) {
+            // A segment named `segmentName` exists within the table named `tableName`
+            return true;
+        }
+        if (res.statusLine.getStatusCode() == 404) {
+            // There is no such segment named `segmentName` within the table named `tableName`
+            // (or the table named `tableName` does not exist)
+            return false;
+        }
+
+        // Received an unexpected status code
+        throw new PinotControllerApiException(res.responseBody);
+    }
+
+    /**
+     * Deletes a segment from a table.
+     *
+     * @param tableName   Target table's name
+     * @param segmentName Identifies the segment to delete
+     * @throws IOException
+     */
+    public void deleteSegment(String tableName, String segmentName) throws IOException {
+        ApiResponse res = this.delete(String.format("/tables/%s/%s", tableName, segmentName));
+
+        if (res.statusLine.getStatusCode() != 200) {
+            LOG.error("Could not delete segment {} from table {}. Pinot controller returned: {}", tableName, segmentName, res.responseBody);
+            throw new PinotControllerApiException(res.responseBody);
+        }
+    }
+
+    /**
+     * Fetches a Pinot table's schema via the Pinot controller API.
+     *
+     * @param tableName Target table's name
+     * @return Pinot table schema
+     * @throws IOException
+     */
+    public Schema getSchema(String tableName) throws IOException {
+        Schema schema;
+        ApiResponse res = this.get(String.format("/tables/%s/schema", tableName));
+        LOG.info("Get schema request for table {} returned {}", tableName, res.responseBody);
+
+        if (res.statusLine.getStatusCode() != 200) {
+            throw new PinotControllerApiException(res.responseBody);
+        }
+
+        try {
+            schema = JsonUtils.stringToObject(res.responseBody, Schema.class);
+        } catch (Exception e) {
+            throw new IllegalStateException("Caught exception while reading schema from Pinot Controller's response: " + res.responseBody, e);
+        }
+        LOG.info("Retrieved schema: {}", schema.toSingleLineJsonString());
+        return schema;
+    }
+
+    /**
+     * Fetches a Pinot table's configuration via the Pinot controller API.
+     *
+     * @param tableName Target table's name
+     * @return Pinot table configuration
+     * @throws IOException
+     */
+    public TableConfig getTableConfig(String tableName) throws IOException {
+        TableConfig tableConfig;
+        ApiResponse res = this.get(String.format("/tables/%s", tableName));
+        LOG.info("Get table config request for table {} returned {}", tableName, res.responseBody);
+
+        try {
+            String tableConfigAsJson = JsonUtils.stringToJsonNode(res.responseBody).get("OFFLINE").toString();
+            tableConfig = JsonUtils.stringToObject(tableConfigAsJson, TableConfig.class);
+        } catch (Exception e) {
+            throw new IllegalStateException("Caught exception while reading table config from Pinot Controller's response: " + res.responseBody, e);
+        }
+        LOG.info("Retrieved table config: {}", tableConfig.toJsonString());

Review comment:
       now using debug instead of info all over the place

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());

Review comment:
       now using debug instead of info all over the place




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

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



[GitHub] [bahir-flink] eskabetxe commented on pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
eskabetxe commented on pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#issuecomment-814829059


   @mschroederi thanks for your contribution
   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.

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594305539



##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.util.TestLogger;
+import org.apache.pinot.spi.config.table.*;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.Wait;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Base class for PinotSink e2e tests
+ */
+@Testcontainers
+public class PinotTestBase extends TestLogger implements Serializable {

Review comment:
       really good question -> removed




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594325729



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {

Review comment:
       `InterruptedException | ExecutionException` possibly thrown when trying to resolve the `Future`




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r592795877



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,428 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        List<PinotSinkGlobalCommittable> committablesToRetry = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+
+            if (commitStatus.getMissingSegmentNames().isEmpty()) {
+                // All segments were already committed. Thus, we do not need to retry the commit.
+                continue;
+            }
+
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+            committablesToRetry.add(globalCommittable);
+        }
+
+        return committablesToRetry;
+    }
+
+    /**
+     * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable}
+     * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s.
+     *
+     * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}
+     * @return Global committer committable
+     */
+    @Override
+    public PinotSinkGlobalCommittable combine(List<PinotSinkCommittable> committables) {
+        List<String> dataFilePaths = new ArrayList<>();
+        long minTimestamp = Long.MAX_VALUE;
+        long maxTimestamp = Long.MIN_VALUE;
+
+        // Extract all data file paths and the overall minimum and maximum timestamps
+        // from all committables
+        for (PinotSinkCommittable committable : committables) {
+            dataFilePaths.add(committable.getDataFilePath());
+            minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp());
+            maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp());
+        }
+
+        LOG.info("Combined {} committables into one global committable", committables.size());
+        return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp);
+    }
+
+    /**
+     * Copies data files from shared filesystem to the local filesystem, generates segments with names
+     * according to the segment naming schema and finally pushes the segments to the Pinot cluster.
+     * Before pushing a segment it is checked whether there already exists a segment with that name
+     * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted.
+     *
+     * @param globalCommittables List of global committables
+     * @return Global committables whose commit failed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> commit(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+        Schema tableSchema = controllerApi.getSchema(this.tableName);
+        TableConfig tableConfig = controllerApi.getTableConfig(this.tableName);
+
+        // List of failed global committables that can be retried later on
+        List<PinotSinkGlobalCommittable> failedCommits = new ArrayList<>();
+
+        for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) {
+            // Make sure to remove all previously committed segments in globalCommittable
+            // when recovering from failure
+            CommitStatus commitStatus = this.getCommitStatus(globalCommittable);
+            for (String existingSegment : commitStatus.getExistingSegmentNames()) {
+                // Some but not all segments were already committed. As we cannot assure the data
+                // files containing the same data as originally when recovering from failure,
+                // we delete the already committed segments in order to recommit them later on.
+                controllerApi.deleteSegment(tableName, existingSegment);
+            }
+
+            // We use a thread pool in order to parallelize the segment creation and segment upload
+            ExecutorService pool = Executors.newCachedThreadPool();
+            Set<Future<Boolean>> resultFutures = new HashSet<>();
+
+            // Commit all segments in globalCommittable
+            int sequenceId = 0;
+            for (String dataFilePath : globalCommittable.getDataFilePaths()) {
+                // Get segment names with increasing sequenceIds
+                String segmentName = this.getSegmentName(globalCommittable, sequenceId++);
+                // Segment committer handling the whole commit process for a single segment
+                Callable<Boolean> segmentCommitter = new SegmentCommitter(
+                        this.pinotControllerHost, this.pinotControllerPort, this.tempDirPrefix,
+                        this.fsAdapter, dataFilePath, segmentName, tableSchema, tableConfig,
+                        this.timeColumnName, this.segmentTimeUnit
+                );
+                // Submits the segment committer to the thread pool
+                resultFutures.add(pool.submit(segmentCommitter));
+            }
+
+            try {
+                for (Future<Boolean> wasSuccessful : resultFutures) {
+                    // In case any of the segment commits wasn't successful we mark the whole
+                    // globalCommittable as failed
+                    if (!wasSuccessful.get()) {
+                        failedCommits.add(globalCommittable);
+                        // Once any of the commits failed, we do not need to check the remaining
+                        // ones, as we try to commit the globalCommittable next time
+                        break;
+                    }
+                }
+            } catch (Exception e) {
+                // In case of an exception mark the whole globalCommittable as failed
+                failedCommits.add(globalCommittable);
+                LOG.error(e.getMessage());
+                e.printStackTrace();
+            }
+        }
+
+        // Return failed commits so that they can be retried later on
+        return failedCommits;
+    }
+
+    /**
+     * Empty method.
+     */
+    @Override
+    public void endOfInput() {
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Helper method for generating segment names using the segment name generator.
+     *
+     * @param globalCommittable Global committable the segment name shall be generated from
+     * @param sequenceId        Incrementing counter
+     * @return generated segment name
+     */
+    private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) {
+        return this.segmentNameGenerator.generateSegmentName(sequenceId, globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp());
+    }
+
+    /**
+     * Evaluates the status of already uploaded segments by requesting segment metadata from the
+     * Pinot controller.
+     *
+     * @param globalCommittable Global committable whose commit status gets evaluated
+     * @return Commit status
+     * @throws IOException
+     */
+    private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);
+
+        List<String> existingSegmentNames = new ArrayList<>();
+        List<String> missingSegmentNames = new ArrayList<>();
+
+        // For all segment names that will be used to submit new segments, check whether the segment
+        // name already exists for the target table
+        for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) {
+            String segmentName = this.getSegmentName(globalCommittable, sequenceId);
+            if (controllerApi.tableHasSegment(this.tableName, segmentName)) {
+                // Segment name already exists
+                existingSegmentNames.add(segmentName);
+            } else {
+                // Segment name does not exist yet
+                missingSegmentNames.add(segmentName);
+            }
+        }
+
+        return new CommitStatus(existingSegmentNames, missingSegmentNames);
+    }
+
+    /**
+     * Wrapper for existing and missing segments in the Pinot cluster.
+     */
+    static class CommitStatus {
+        private final List<String> existingSegmentNames;
+        private final List<String> missingSegmentNames;
+
+        public CommitStatus(List<String> existingSegmentNames, List<String> missingSegmentNames) {
+            this.existingSegmentNames = existingSegmentNames;
+            this.missingSegmentNames = missingSegmentNames;
+        }
+
+        public List<String> getExistingSegmentNames() {
+            return existingSegmentNames;
+        }
+
+        public List<String> getMissingSegmentNames() {
+            return missingSegmentNames;
+        }
+    }
+
+    /**
+     * Helper class for committing a single segment. Downloads a data file from the shared filesystem,
+     * generates a segment from the data file and uploads segment to the Pinot controller.
+     */
+    static class SegmentCommitter implements Callable<Boolean> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class);
+
+        final String pinotControllerHost;
+        final String pinotControllerPort;
+        final String tempDirPrefix;
+        final FileSystemAdapter fsAdapter;
+        final String dataFilePath;
+        final String segmentName;
+        final Schema tableSchema;
+        final TableConfig tableConfig;
+        final String timeColumnName;
+        final TimeUnit segmentTimeUnit;
+
+        /**
+         * @param pinotControllerHost Host of the Pinot controller
+         * @param pinotControllerPort Port of the Pinot controller
+         * @param fsAdapter           Filesystem adapter used to load data files from the shared file system
+         * @param dataFilePath        Data file to load from the shared file system
+         * @param segmentName         Name of the segment to create and commit
+         * @param tableSchema         Pinot table schema
+         * @param tableConfig         Pinot table config
+         * @param timeColumnName      Name of the column containing the timestamp
+         * @param segmentTimeUnit     Unit of the time column
+         */
+        public SegmentCommitter(String pinotControllerHost, String pinotControllerPort, String tempDirPrefix, FileSystemAdapter fsAdapter, String dataFilePath, String segmentName, Schema tableSchema, TableConfig tableConfig, String timeColumnName, TimeUnit segmentTimeUnit) {
+            this.pinotControllerHost = pinotControllerHost;
+            this.pinotControllerPort = pinotControllerPort;
+            this.tempDirPrefix = tempDirPrefix;
+            this.fsAdapter = fsAdapter;
+            this.dataFilePath = dataFilePath;
+            this.segmentName = segmentName;
+            this.tableSchema = tableSchema;
+            this.tableConfig = tableConfig;
+            this.timeColumnName = timeColumnName;
+            this.segmentTimeUnit = segmentTimeUnit;
+        }
+
+        /**
+         * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment
+         * and finally uploads the segment to the Pinot controller
+         *
+         * @return True if the commit succeeded
+         */
+        @Override
+        public Boolean call() {
+            try {
+                // Download data file from the shared filesystem
+                LOG.info("Downloading data file {} from shared file system...", dataFilePath);
+                File segmentData = fsAdapter.copyToLocalFile(dataFilePath);
+                LOG.info("Successfully downloaded data file {} from shared file system", dataFilePath);
+
+                File segmentFile = Files.createTempDirectory(this.tempDirPrefix).toFile();
+                LOG.info("Creating segment in " + segmentFile.getAbsolutePath());
+
+                // Creates a segment with name `segmentName` in `segmentFile`
+                this.generateSegment(segmentData, segmentFile, true);
+
+                // Uploads the recently created segment to the Pinot controller
+                this.uploadSegment(segmentFile);
+
+                // Commit successful
+                return true;
+            } catch (IOException e) {
+                e.printStackTrace();
+                LOG.error(e.getMessage());
+
+                // Commit failed
+                return false;
+            }
+        }
+
+        /**
+         * Creates a segment from the given parameters.
+         * This method was adapted from {@link org.apache.pinot.tools.admin.command.CreateSegmentCommand}.
+         *
+         * @param dataFile                  File containing the JSON data
+         * @param outDir                    Segment target path
+         * @param _postCreationVerification Verify segment after generation
+         */
+        public void generateSegment(File dataFile, File outDir, Boolean _postCreationVerification) {
+            SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(tableConfig, tableSchema);
+            segmentGeneratorConfig.setSegmentName(segmentName);
+            segmentGeneratorConfig.setSegmentTimeUnit(segmentTimeUnit);
+            segmentGeneratorConfig.setTimeColumnName(timeColumnName);
+            segmentGeneratorConfig.setInputFilePath(dataFile.getPath());
+            segmentGeneratorConfig.setFormat(FileFormat.JSON);
+            segmentGeneratorConfig.setOutDir(outDir.getPath());
+            segmentGeneratorConfig.setTableName(tableConfig.getTableName());
+
+            try {
+                SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+                driver.init(segmentGeneratorConfig);
+                driver.build();
+                File indexDir = new File(outDir, segmentName);
+                LOG.info("Successfully created segment: {} in directory: {}", segmentName, indexDir);
+                if (_postCreationVerification) {
+                    LOG.info("Verifying the segment by loading it");
+                    ImmutableSegment segment = ImmutableSegmentLoader.load(indexDir, ReadMode.mmap);
+                    LOG.info("Successfully loaded segment: {} of size: {} bytes", segmentName,
+                            segment.getSegmentSizeBytes());
+                    segment.destroy();
+                }
+            } catch (Exception e) {
+                e.printStackTrace();

Review comment:
       done




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

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



[GitHub] [bahir-flink] fapaul commented on pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
fapaul commented on pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#issuecomment-802988116


   @mschroederi can you please close the fixed discussions? ;)


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

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



[GitHub] [bahir-flink] fapaul commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
fapaul commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r595812015



##########
File path: flink-connector-pinot/README.md
##########
@@ -0,0 +1,126 @@
+# Flink Pinot Connector
+
+This connector provides a sink to [Apache Pinot](http://pinot.apache.org/)™.  
+To use this connector, add the following dependency to your project:
+
+    <dependency>
+      <groupId>org.apache.bahir</groupId>
+      <artifactId>flink-connector-pinot_2.11</artifactId>
+      <version>1.1-SNAPSHOT</version>
+    </dependency>
+
+*Version Compatibility*: This module is compatible with Pinot 0.6.0.
+
+Note that the streaming connectors are not part of the binary distribution of Flink. You need to link them into your job jar for cluster execution.
+See how to link with them for cluster execution [here](https://ci.apache.org/projects/flink/flink-docs-release-1.2/dev/linking.html).
+
+The sink class is called `PinotSink`.
+
+## Usage
+```java
+StreamExecutionEnvironment env = ...
+// Checkpointing needs to be enabled when executing in STREAMING mode
+        env.enableCheckpointing(long interval);
+
+        DataStream<PinotRow> dataStream = ...
+        PinotSink pinotSink = new PinotSink.Builder<PinotRow>(String pinotControllerHost, String pinotControllerPort, String tableName)
+
+        // Serializes a PinotRow to JSON format
+        .withJsonSerializer(JsonSerializer<PinotRow> jsonSerializer)
+
+        // Extracts the timestamp from a PinotRow
+        .withEventTimeExtractor(EventTimeExtractor<IN> eventTimeExtractor)
+
+        // Defines the segment name generation via the predefined SimpleSegmentNameGenerator
+        // Exemplary segment name: tableName_minTimestamp_maxTimestamp_segmentNamePostfix_0
+        .withSimpleSegmentNameGenerator(String tableName, String segmentNamePostfix)
+
+        // Use a custom segment name generator if the SimpleSegmentNameGenerator does not work for your use case
+        .withSegmentNameGenerator(SegmentNameGenerator segmentNameGenerator)
+
+        // Use the local filesystem to share committables across subTasks
+        // CAUTION: Use only if all subTasks run on the same node with access to the local filesystem
+        .withLocalFileSystemAdapter()
+
+        // Use a custom filesystem adapter. 
+        // CAUTION: Make sure all nodes your Flink app runs on can access the shared filesystem via the provided FileSystemAdapter
+        .withFileSystemAdapter(FileSystemAdapter fsAdapter)
+
+        // Defines the size of the Pinot segments
+        .withMaxRowsPerSegment(int maxRowsPerSegment)
+
+        // Prefix within the local filesystem's temp directory used for storing intermediate files
+        .withTempDirectoryPrefix(String tempDirPrefix)
+        
+        // Number of threads used in the `PinotSinkGlobalCommitter` to commit a batch of segments
+        // Optional - Default is 4
+        .withNumCommitThreads(int numCommitThreads)
+
+        // Builds the PinotSink
+        .build()
+        dataStream.addSink(pinotSink);
+```
+
+## Options
+| Option                 | Description                                                                      |
+| ---------------------- | -------------------------------------------------------------------------------- | 
+| `pinotControllerHost`  | Host of the Pinot controller                                                     |
+| `pinotControllerPort`  | Port of the Pinot controller                                                     |
+| `tableName`            | Target Pinot table's name                                                        |
+| `maxRowsPerSegment`    | Maximum number of rows to be stored within a Pinot segment                       |
+| `tempDirPrefix`         | Prefix for temp directories used                                                  |
+| `jsonSerializer`       | Serializer used to convert elements to JSON                                      |
+| `eventTimeExtractor`   | Defines the way event times are extracted from received objects                   |
+| `segmentNameGenerator` | Pinot segment name generator                                                     |
+| `fsAdapter`            | Filesystem adapter used to save files for sharing files across nodes               |
+| `numCommitThreads`     | Number of threads used in the `PinotSinkGlobalCommitter` for committing segments |
+
+## Architecture

Review comment:
       I would reorder this section a bit.
   1. Architecture
   2. PinotSinkWriter 
   3. PinotGlobalCommitter
   4. DeliveryGuarantees
   5. Options
   6. Usage

##########
File path: flink-connector-pinot/pom.xml
##########
@@ -0,0 +1,225 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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/maven-v4_0_0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.bahir</groupId>
+        <artifactId>bahir-flink-parent_2.11</artifactId>
+        <version>1.1-SNAPSHOT</version>
+        <relativePath>..</relativePath>
+    </parent>
+
+    <artifactId>flink-connector-pinot_2.11</artifactId>
+    <name>flink-connector-pinot</name>
+
+
+    <packaging>jar</packaging>
+
+    <!-- Allow users to pass custom connector versions -->
+    <properties>
+        <pinot.version>0.6.0</pinot.version>
+
+        <testcontainers.version>1.15.2</testcontainers.version>

Review comment:
       I do not how other bahir projects handle this but is it important that users can overwrite the test container version? I would see tests as internal.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerClient.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerClient implements Closeable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerClient.class);
+    private final PinotControllerHttpClient httpClient;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerClient(String controllerHost, String controllerPort) {
+        httpClient = new PinotControllerHttpClient(controllerHost, controllerPort);

Review comment:
       In the constructor you are supposed to use `this.` :) 

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerHttpClient.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.*;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helpers to interact with the Pinot controller via its public API.
+ */
+public class PinotControllerHttpClient implements Closeable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotControllerHttpClient.class);
+    protected final String controllerHostPort;
+    protected final CloseableHttpClient httpClient;
+
+    /**
+     * @param controllerHost Pinot controller's host
+     * @param controllerPort Pinot controller's port
+     */
+    public PinotControllerHttpClient(String controllerHost, String controllerPort) {
+        checkNotNull(controllerHost);
+        checkNotNull(controllerPort);
+        controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort);
+        httpClient = HttpClients.createDefault();
+    }
+
+    /**
+     * Issues a request to the Pinot controller API.
+     *
+     * @param request Request to issue
+     * @return Api response
+     * @throws IOException
+     */
+    private ApiResponse execute(HttpRequestBase request) throws IOException {
+        ApiResponse result;
+
+        try (CloseableHttpResponse response = httpClient.execute(request)) {
+            String body = EntityUtils.toString(response.getEntity());
+            result = new ApiResponse(response.getStatusLine(), body);
+        }
+
+        return result;
+    }
+
+    /**
+     * Issues a POST request to the Pinot controller API.
+     *
+     * @param path Path to POST to
+     * @param body Request's body
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse post(String path, String body) throws IOException {
+        HttpPost httppost = new HttpPost(this.controllerHostPort + path);
+        httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON));
+        LOG.debug("Posting string entity {} to {}", body, path);
+        return this.execute(httppost);
+    }
+
+    /**
+     * Issues a GET request to the Pinot controller API.
+     *
+     * @param path Path to GET from
+     * @return API response
+     * @throws IOException
+     */
+    protected ApiResponse get(String path) throws IOException {

Review comment:
       private

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/filesystem/LocalFileSystemAdapter.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.flink.streaming.connectors.pinot.filesystem;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The LocalFileSystemAdapter is used when sharing files via the local filesystem.
+ * Keep in mind that using this FileSystemAdapter requires running the Flink app on a single node.
+ */
+public class LocalFileSystemAdapter extends FileSystemAdapter {

Review comment:
       I would move this class to the test package and also remove this option from the sink builder because it is a highly unlikely setting.

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/filesystem/FileSystemAdapter.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.flink.streaming.connectors.pinot.filesystem;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+
+/**
+ * Defines the interaction with a shared filesystem. The shared filesystem must be accessible from all
+ * nodes within the cluster than run a partition of the {@link org.apache.flink.streaming.connectors.pinot.PinotSink}.
+ */
+public abstract class FileSystemAdapter implements Serializable {

Review comment:
       Can this be an interface?

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkCommittableSerializer.java
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.flink.streaming.connectors.pinot.serializer;
+
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+
+import java.io.*;
+
+/**
+ * Serializer for {@link PinotSinkCommittable}
+ */
+public class PinotSinkCommittableSerializer implements SimpleVersionedSerializer<PinotSinkCommittable> {
+
+    private static final int CURRENT_VERSION = 1;
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(PinotSinkCommittable pinotSinkCommittable) throws IOException {
+        try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+             DataOutputStream out = new DataOutputStream(baos)) {
+            out.writeLong(pinotSinkCommittable.getMinTimestamp());
+            out.writeLong(pinotSinkCommittable.getMaxTimestamp());
+            out.writeUTF(pinotSinkCommittable.getDataFilePath());
+            out.flush();
+            return baos.toByteArray();
+        }
+    }
+
+    @Override
+    public PinotSinkCommittable deserialize(int version, byte[] serialized) throws IOException {
+        switch (version) {
+            case 1:
+                return deserializeV1(serialized);
+            default:
+                throw new IOException("Unrecognized version or corrupt state: " + version);

Review comment:
       ```suggestion
                   throw new IllegalStateException("Unrecognized version or corrupt state: " + version);
   ```

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkGlobalCommittableSerializer.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.flink.streaming.connectors.pinot.serializer;
+
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommittable;
+
+import java.io.*;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Serializer for {@link PinotSinkGlobalCommittable}
+ */
+public class PinotSinkGlobalCommittableSerializer implements SimpleVersionedSerializer<PinotSinkGlobalCommittable> {
+
+    private static final int CURRENT_VERSION = 1;
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(PinotSinkGlobalCommittable pinotSinkGlobalCommittable) throws IOException {
+        try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+             DataOutputStream out = new DataOutputStream(baos)) {
+            out.writeLong(pinotSinkGlobalCommittable.getMinTimestamp());
+            out.writeLong(pinotSinkGlobalCommittable.getMaxTimestamp());
+
+            int size = pinotSinkGlobalCommittable.getDataFilePaths().size();
+            out.writeInt(size);
+            for (String dataFilePath : pinotSinkGlobalCommittable.getDataFilePaths()) {
+                out.writeUTF(dataFilePath);
+            }
+            out.flush();
+            return baos.toByteArray();
+        }
+    }
+
+    @Override
+    public PinotSinkGlobalCommittable deserialize(int version, byte[] serialized) throws IOException {
+        switch (version) {
+            case 1:
+                return deserializeV1(serialized);
+            default:
+                throw new IOException("Unrecognized version or corrupt state: " + version);

Review comment:
       ```suggestion
                   throw new IllegalStateException("Unrecognized version or corrupt state: " + version);
   ```

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import com.google.common.collect.Iterables;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Accepts incoming elements and creates {@link PinotSinkCommittable}s out of them on request.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotSinkWriter<IN> implements SinkWriter<IN, PinotSinkCommittable, Void> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkWriter.class);
+
+    private final int maxRowsPerSegment;
+    private final EventTimeExtractor<IN> eventTimeExtractor;
+    private final JsonSerializer<IN> jsonSerializer;
+
+    private final List<PinotWriterSegment<IN>> activeSegments;
+    private final FileSystemAdapter fsAdapter;
+
+    private final int subtaskId;
+
+    /**
+     * @param subtaskId          Subtask id provided by Flink
+     * @param maxRowsPerSegment  Maximum number of rows to be stored within a Pinot segment
+     * @param eventTimeExtractor Defines the way event times are extracted from received objects
+     * @param jsonSerializer     Serializer used to convert elements to JSON
+     * @param fsAdapter          Filesystem adapter used to save files for sharing files across nodes
+     */
+    public PinotSinkWriter(int subtaskId, int maxRowsPerSegment,
+                           EventTimeExtractor<IN> eventTimeExtractor,
+                           JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        this.subtaskId = subtaskId;
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.eventTimeExtractor = checkNotNull(eventTimeExtractor);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.activeSegments = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements from an upstream tasks and writes them into {@link PinotWriterSegment}
+     *
+     * @param element Object from upstream task
+     * @param context SinkWriter context
+     * @throws IOException
+     */
+    @Override
+    public void write(IN element, Context context) throws IOException {
+        final PinotWriterSegment<IN> inProgressSegment = getOrCreateInProgressSegment();
+        inProgressSegment.write(element, eventTimeExtractor.getEventTime(element, context));
+    }
+
+    /**
+     * Creates {@link PinotSinkCommittable}s from elements previously received via {@link #write}.
+     * If flush is set, all {@link PinotWriterSegment}s are transformed into
+     * {@link PinotSinkCommittable}s. If flush is not set, only currently non-active
+     * {@link PinotSinkCommittable}s are transformed into {@link PinotSinkCommittable}s.
+     * To convert a {@link PinotWriterSegment} into a {@link PinotSinkCommittable} the data gets
+     * written to the shared filesystem. Moreover, minimum and maximum timestamps are identified.
+     * Finally, all {@link PinotWriterSegment}s transformed into {@link PinotSinkCommittable}s are
+     * removed from {@link #activeSegments}.
+     *
+     * @param flush Flush all currently known elements into the {@link PinotSinkCommittable}s
+     * @return List of {@link PinotSinkCommittable} to process in {@link org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter}
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkCommittable> prepareCommit(boolean flush) throws IOException {
+        // Identify segments to commit. If the flush argument is set all segments shall be committed.
+        // Otherwise, take only those PinotWriterSegments that do not accept any more elements.
+        List<PinotWriterSegment<IN>> segmentsToCommit = activeSegments.stream()
+                .filter(s -> flush || !s.acceptsElements())
+                .collect(Collectors.toList());
+        LOG.debug("Identified {} segments to commit [subtaskId={}]", segmentsToCommit.size(), subtaskId);
+
+        LOG.debug("Creating committables... [subtaskId={}]", subtaskId);
+        List<PinotSinkCommittable> committables = new ArrayList<>();
+        for (final PinotWriterSegment<IN> segment : segmentsToCommit) {
+            committables.add(segment.prepareCommit());
+        }
+        LOG.debug("Created {} committables [subtaskId={}]", committables.size(), subtaskId);
+
+        // Remove all PinotWriterSegments that will be emitted within the committables.
+        activeSegments.removeAll(segmentsToCommit);
+        return committables;
+    }
+
+    /**
+     * Gets the {@link PinotWriterSegment} still accepting elements or creates a new one.
+     *
+     * @return {@link PinotWriterSegment} accepting at least one more element
+     */
+    private PinotWriterSegment<IN> getOrCreateInProgressSegment() {
+        final PinotWriterSegment<IN> latestSegment = Iterables.getLast(activeSegments, null);
+        if (latestSegment == null || !latestSegment.acceptsElements()) {
+            final PinotWriterSegment<IN> inProgressSegment = new PinotWriterSegment<>(maxRowsPerSegment, jsonSerializer, fsAdapter);
+            activeSegments.add(inProgressSegment);
+            return inProgressSegment;
+        }
+        return latestSegment;
+    }
+
+    /**
+     * As we do not need to save any information in snapshots,
+     * this method always returns an empty ArrayList.
+     *
+     * @return always an empty ArrayList
+     */
+    @Override
+    public List<Void> snapshotState() {

Review comment:
       I would expect that we have to save the path to the active files on every checkpoint.

##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestHelper.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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException;
+import org.apache.pinot.client.*;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * Helper class ot interact with the Pinot controller and broker in the e2e tests
+ */
+public class PinotTestHelper implements Closeable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotTestHelper.class);
+    private final String host;
+    private final String brokerPort;
+    private final PinotControllerHttpClient httpClient;
+
+    /**
+     * @param host           Host the Pinot controller and broker are accessible at
+     * @param controllerPort The Pinot controller's external port at {@code host}
+     * @param brokerPort     A Pinot broker's external port at {@code host}
+     */
+    public PinotTestHelper(String host, String controllerPort, String brokerPort) {
+        this.host = host;
+        this.brokerPort = brokerPort;
+        httpClient = new PinotControllerHttpClient(host, controllerPort);

Review comment:
       ```suggestion
           this.httpClient = new PinotControllerHttpClient(host, controllerPort);
   ```




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

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



[GitHub] [bahir-flink] mschroederi commented on pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#issuecomment-803687008


   > Thanks for all your efforts, you did a great job!
   > 
   > Besides my final comment, this PR looks ready to merge from my side. Now we only need to wait for the final approval of Bahir committer or PMC.
   > 
   > Please squash all your changes in a single commit, to make the committer's life easier ;)
   
   @fapaul Thanks for your support during this whole process!


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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594689711



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import com.google.common.collect.Iterables;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Accepts incoming elements and creates {@link PinotSinkCommittable}s out of them on request.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotSinkWriter<IN> implements SinkWriter<IN, PinotSinkCommittable, Void> {
+
+    private static final Logger LOG = LoggerFactory.getLogger("PinotSinkWriter");
+
+    private final int maxRowsPerSegment;
+    private EventTimeExtractor<IN> eventTimeExtractor;
+    private final String tempDirPrefix;
+    private final JsonSerializer<IN> jsonSerializer;
+
+    private final List<PinotWriterSegment<IN>> activeSegments;
+    private final FileSystemAdapter fsAdapter;
+
+    private final int subtaskId;
+
+    /**
+     * @param subtaskId          Subtask id provided by Flink
+     * @param maxRowsPerSegment  Maximum number of rows to be stored within a Pinot segment
+     * @param eventTimeExtractor Defines the way event times are extracted from received objects
+     * @param tempDirPrefix      Prefix for temp directories used
+     * @param jsonSerializer     Serializer used to convert elements to JSON
+     * @param fsAdapter          Filesystem adapter used to save files for sharing files across nodes
+     */
+    public PinotSinkWriter(int subtaskId, int maxRowsPerSegment, EventTimeExtractor<IN> eventTimeExtractor, String tempDirPrefix, JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        this.subtaskId = subtaskId;
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.eventTimeExtractor = checkNotNull(eventTimeExtractor);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.activeSegments = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements from an upstream tasks and writes them into {@link PinotWriterSegment}
+     *
+     * @param element Object from upstream task
+     * @param context SinkWriter context
+     * @throws IOException
+     */
+    @Override
+    public void write(IN element, Context context) throws IOException {
+        final PinotWriterSegment<IN> inProgressSegment = this.getOrCreateInProgressSegment();
+        inProgressSegment.write(element, this.eventTimeExtractor.getEventTime(element, context));
+    }
+
+    /**
+     * Creates {@link PinotSinkCommittable}s from elements received via {@link #write}
+     *
+     * @param flush Flush all currently known elements into the {@link PinotSinkCommittable}s
+     * @return List of {@link PinotSinkCommittable} to process in {@link org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter}
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkCommittable> prepareCommit(boolean flush) throws IOException {
+        List<PinotWriterSegment<IN>> segmentsToCommit = this.activeSegments.stream()
+                .filter(s -> flush || !s.acceptsElements())
+                .collect(Collectors.toList());
+        LOG.info("Identified {} segments to commit [subtaskId={}]", segmentsToCommit.size(), this.subtaskId);
+
+        LOG.info("Creating committables... [subtaskId={}]", subtaskId);
+        List<PinotSinkCommittable> committables = new ArrayList<>();
+        for (final PinotWriterSegment<IN> segment : segmentsToCommit) {
+            committables.add(segment.prepareCommit());
+        }
+        LOG.info("Created {} committables [subtaskId={}]", committables.size(), subtaskId);
+
+        this.activeSegments.removeAll(segmentsToCommit);
+        return committables;
+    }
+
+    /**
+     * Gets the {@link PinotWriterSegment} still accepting elements or creates a new one.
+     *
+     * @return {@link PinotWriterSegment} accepting at least one more element
+     */
+    private PinotWriterSegment<IN> getOrCreateInProgressSegment() {
+        final PinotWriterSegment<IN> latestSegment = Iterables.getLast(this.activeSegments, null);
+        if (latestSegment == null || !latestSegment.acceptsElements()) {
+            final PinotWriterSegment<IN> inProgressSegment = new PinotWriterSegment<>(this.maxRowsPerSegment, this.tempDirPrefix, this.jsonSerializer, this.fsAdapter);
+            this.activeSegments.add(inProgressSegment);
+            return inProgressSegment;
+        }
+        return latestSegment;
+    }
+
+    /**
+     * Snapshots the current state to be stored within a checkpoint. As we do not need to save any
+     * information in snapshots, this method always returns an empty ArrayList.
+     *
+     * @return always an empty ArrayList
+     */
+    @Override
+    public List<Void> snapshotState() {
+        return new ArrayList<>();
+    }
+
+    /**
+     * Empty method, as we do not open any connections.
+     */
+    @Override
+    public void close() {

Review comment:
       No temp files anymore in the SinkWriter




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r595083857



##########
File path: flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/EmulatedPinotSinkTest.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.flink.streaming.connectors.pinot;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.filesystem.LocalFileSystemAdapter;
+import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator;
+import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator;
+import org.apache.pinot.client.ResultSet;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * E2e tests for Pinot Sink using BATCH and STREAMING execution mode
+ */
+public class EmulatedPinotSinkTest extends PinotTestBase {
+
+    /**
+     * Tests the BATCH execution of the {@link PinotSink}.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testBatchSink() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.BATCH);
+        env.setParallelism(2);
+
+        List<SingleColumnTableRow> data = getTestData(12);
+        this.setupDataStream(env, data);
+
+        // Run
+        env.execute();
+
+        TimeUnit.SECONDS.sleep(5);

Review comment:
       The issue here is that Pinot might take some time in order to index and distribute the recently uploaded segments (independent from Streaming or Batching mode). 
   In order to remove complexity from the actual testing code, I've now implemented a `checkForDataInPinotWithRetry` method which uses a linear retry backoff delay to regularly call `checkForDataInPinot` up to a given timeout.




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594309067



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);

Review comment:
       see above -> done

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);

Review comment:
       done

##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.flink.streaming.connectors.pinot.committer;
+
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.streaming.connectors.pinot.PinotControllerApi;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.name.SegmentNameGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.tools.admin.command.UploadSegmentCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter},
+ * generates segments and pushed them to the Pinot controller.
+ * Note: We use a custom multithreading approach to parallelize the segment creation and upload to
+ * overcome the performance limitations resulting from using a {@link GlobalCommitter} always
+ * running at a parallelism of 1.
+ */
+public class PinotSinkGlobalCommitter implements GlobalCommitter<PinotSinkCommittable, PinotSinkGlobalCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class);
+
+    private final String pinotControllerHost;
+    private final String pinotControllerPort;
+    private final String tableName;
+    private final SegmentNameGenerator segmentNameGenerator;
+    private final String tempDirPrefix;
+    private final FileSystemAdapter fsAdapter;
+    private final String timeColumnName;
+    private final TimeUnit segmentTimeUnit;
+
+    /**
+     * @param pinotControllerHost  Host of the Pinot controller
+     * @param pinotControllerPort  Port of the Pinot controller
+     * @param tableName            Target table's name
+     * @param segmentNameGenerator Pinot segment name generator
+     * @param fsAdapter            Adapter for interacting with the shared file system
+     * @param timeColumnName       Name of the column containing the timestamp
+     * @param segmentTimeUnit      Unit of the time column
+     */
+    public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, String tableName, SegmentNameGenerator segmentNameGenerator, String tempDirPrefix, FileSystemAdapter fsAdapter, String timeColumnName, TimeUnit segmentTimeUnit) {
+        this.pinotControllerHost = checkNotNull(pinotControllerHost);
+        this.pinotControllerPort = checkNotNull(pinotControllerPort);
+        this.tableName = checkNotNull(tableName);
+        this.segmentNameGenerator = checkNotNull(segmentNameGenerator);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.timeColumnName = checkNotNull(timeColumnName);
+        this.segmentTimeUnit = checkNotNull(segmentTimeUnit);
+    }
+
+    /**
+     * Identifies global committables that need to be re-committed from a list of recovered committables.
+     *
+     * @param globalCommittables List of global committables that are checked for required re-commit
+     * @return List of global committable that need to be re-committed
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkGlobalCommittable> filterRecoveredCommittables(List<PinotSinkGlobalCommittable> globalCommittables) throws IOException {
+        PinotControllerApi controllerApi = new PinotControllerApi(this.pinotControllerHost, this.pinotControllerPort);

Review comment:
       done




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r594689440



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import com.google.common.collect.Iterables;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Accepts incoming elements and creates {@link PinotSinkCommittable}s out of them on request.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotSinkWriter<IN> implements SinkWriter<IN, PinotSinkCommittable, Void> {
+
+    private static final Logger LOG = LoggerFactory.getLogger("PinotSinkWriter");
+
+    private final int maxRowsPerSegment;
+    private EventTimeExtractor<IN> eventTimeExtractor;
+    private final String tempDirPrefix;
+    private final JsonSerializer<IN> jsonSerializer;
+
+    private final List<PinotWriterSegment<IN>> activeSegments;
+    private final FileSystemAdapter fsAdapter;
+
+    private final int subtaskId;
+
+    /**
+     * @param subtaskId          Subtask id provided by Flink
+     * @param maxRowsPerSegment  Maximum number of rows to be stored within a Pinot segment
+     * @param eventTimeExtractor Defines the way event times are extracted from received objects
+     * @param tempDirPrefix      Prefix for temp directories used
+     * @param jsonSerializer     Serializer used to convert elements to JSON
+     * @param fsAdapter          Filesystem adapter used to save files for sharing files across nodes
+     */
+    public PinotSinkWriter(int subtaskId, int maxRowsPerSegment, EventTimeExtractor<IN> eventTimeExtractor, String tempDirPrefix, JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        this.subtaskId = subtaskId;
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.eventTimeExtractor = checkNotNull(eventTimeExtractor);
+        this.tempDirPrefix = checkNotNull(tempDirPrefix);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.activeSegments = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements from an upstream tasks and writes them into {@link PinotWriterSegment}
+     *
+     * @param element Object from upstream task
+     * @param context SinkWriter context
+     * @throws IOException
+     */
+    @Override
+    public void write(IN element, Context context) throws IOException {
+        final PinotWriterSegment<IN> inProgressSegment = this.getOrCreateInProgressSegment();
+        inProgressSegment.write(element, this.eventTimeExtractor.getEventTime(element, context));
+    }
+
+    /**
+     * Creates {@link PinotSinkCommittable}s from elements received via {@link #write}
+     *
+     * @param flush Flush all currently known elements into the {@link PinotSinkCommittable}s
+     * @return List of {@link PinotSinkCommittable} to process in {@link org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter}
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkCommittable> prepareCommit(boolean flush) throws IOException {
+        List<PinotWriterSegment<IN>> segmentsToCommit = this.activeSegments.stream()
+                .filter(s -> flush || !s.acceptsElements())
+                .collect(Collectors.toList());
+        LOG.info("Identified {} segments to commit [subtaskId={}]", segmentsToCommit.size(), this.subtaskId);
+
+        LOG.info("Creating committables... [subtaskId={}]", subtaskId);
+        List<PinotSinkCommittable> committables = new ArrayList<>();
+        for (final PinotWriterSegment<IN> segment : segmentsToCommit) {
+            committables.add(segment.prepareCommit());
+        }
+        LOG.info("Created {} committables [subtaskId={}]", committables.size(), subtaskId);
+
+        this.activeSegments.removeAll(segmentsToCommit);
+        return committables;
+    }
+
+    /**
+     * Gets the {@link PinotWriterSegment} still accepting elements or creates a new one.
+     *
+     * @return {@link PinotWriterSegment} accepting at least one more element
+     */
+    private PinotWriterSegment<IN> getOrCreateInProgressSegment() {
+        final PinotWriterSegment<IN> latestSegment = Iterables.getLast(this.activeSegments, null);
+        if (latestSegment == null || !latestSegment.acceptsElements()) {
+            final PinotWriterSegment<IN> inProgressSegment = new PinotWriterSegment<>(this.maxRowsPerSegment, this.tempDirPrefix, this.jsonSerializer, this.fsAdapter);
+            this.activeSegments.add(inProgressSegment);
+            return inProgressSegment;
+        }
+        return latestSegment;
+    }
+
+    /**
+     * Snapshots the current state to be stored within a checkpoint. As we do not need to save any
+     * information in snapshots, this method always returns an empty ArrayList.
+     *
+     * @return always an empty ArrayList
+     */
+    @Override
+    public List<Void> snapshotState() {

Review comment:
       The `PinotWriterSegment` now directly writes to the shared filesystem via `FileSystemAdapter`, so there aren't any temporary local files used anymore in the SinkWriter.
   Files stored on the shared filesystem will be deleted once the commit was successful.




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

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



[GitHub] [bahir-flink] mschroederi commented on a change in pull request #113: Apache Pinot Connector Sink

Posted by GitBox <gi...@apache.org>.
mschroederi commented on a change in pull request #113:
URL: https://github.com/apache/bahir-flink/pull/113#discussion_r597272545



##########
File path: flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.flink.streaming.connectors.pinot.writer;
+
+import com.google.common.collect.Iterables;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable;
+import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor;
+import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer;
+import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Accepts incoming elements and creates {@link PinotSinkCommittable}s out of them on request.
+ *
+ * @param <IN> Type of incoming elements
+ */
+public class PinotSinkWriter<IN> implements SinkWriter<IN, PinotSinkCommittable, Void> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PinotSinkWriter.class);
+
+    private final int maxRowsPerSegment;
+    private final EventTimeExtractor<IN> eventTimeExtractor;
+    private final JsonSerializer<IN> jsonSerializer;
+
+    private final List<PinotWriterSegment<IN>> activeSegments;
+    private final FileSystemAdapter fsAdapter;
+
+    private final int subtaskId;
+
+    /**
+     * @param subtaskId          Subtask id provided by Flink
+     * @param maxRowsPerSegment  Maximum number of rows to be stored within a Pinot segment
+     * @param eventTimeExtractor Defines the way event times are extracted from received objects
+     * @param jsonSerializer     Serializer used to convert elements to JSON
+     * @param fsAdapter          Filesystem adapter used to save files for sharing files across nodes
+     */
+    public PinotSinkWriter(int subtaskId, int maxRowsPerSegment,
+                           EventTimeExtractor<IN> eventTimeExtractor,
+                           JsonSerializer<IN> jsonSerializer, FileSystemAdapter fsAdapter) {
+        this.subtaskId = subtaskId;
+        this.maxRowsPerSegment = maxRowsPerSegment;
+        this.eventTimeExtractor = checkNotNull(eventTimeExtractor);
+        this.jsonSerializer = checkNotNull(jsonSerializer);
+        this.fsAdapter = checkNotNull(fsAdapter);
+        this.activeSegments = new ArrayList<>();
+    }
+
+    /**
+     * Takes elements from an upstream tasks and writes them into {@link PinotWriterSegment}
+     *
+     * @param element Object from upstream task
+     * @param context SinkWriter context
+     * @throws IOException
+     */
+    @Override
+    public void write(IN element, Context context) throws IOException {
+        final PinotWriterSegment<IN> inProgressSegment = getOrCreateInProgressSegment();
+        inProgressSegment.write(element, eventTimeExtractor.getEventTime(element, context));
+    }
+
+    /**
+     * Creates {@link PinotSinkCommittable}s from elements previously received via {@link #write}.
+     * If flush is set, all {@link PinotWriterSegment}s are transformed into
+     * {@link PinotSinkCommittable}s. If flush is not set, only currently non-active
+     * {@link PinotSinkCommittable}s are transformed into {@link PinotSinkCommittable}s.
+     * To convert a {@link PinotWriterSegment} into a {@link PinotSinkCommittable} the data gets
+     * written to the shared filesystem. Moreover, minimum and maximum timestamps are identified.
+     * Finally, all {@link PinotWriterSegment}s transformed into {@link PinotSinkCommittable}s are
+     * removed from {@link #activeSegments}.
+     *
+     * @param flush Flush all currently known elements into the {@link PinotSinkCommittable}s
+     * @return List of {@link PinotSinkCommittable} to process in {@link org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter}
+     * @throws IOException
+     */
+    @Override
+    public List<PinotSinkCommittable> prepareCommit(boolean flush) throws IOException {
+        // Identify segments to commit. If the flush argument is set all segments shall be committed.
+        // Otherwise, take only those PinotWriterSegments that do not accept any more elements.
+        List<PinotWriterSegment<IN>> segmentsToCommit = activeSegments.stream()
+                .filter(s -> flush || !s.acceptsElements())
+                .collect(Collectors.toList());
+        LOG.debug("Identified {} segments to commit [subtaskId={}]", segmentsToCommit.size(), subtaskId);
+
+        LOG.debug("Creating committables... [subtaskId={}]", subtaskId);
+        List<PinotSinkCommittable> committables = new ArrayList<>();
+        for (final PinotWriterSegment<IN> segment : segmentsToCommit) {
+            committables.add(segment.prepareCommit());
+        }
+        LOG.debug("Created {} committables [subtaskId={}]", committables.size(), subtaskId);
+
+        // Remove all PinotWriterSegments that will be emitted within the committables.
+        activeSegments.removeAll(segmentsToCommit);
+        return committables;
+    }
+
+    /**
+     * Gets the {@link PinotWriterSegment} still accepting elements or creates a new one.
+     *
+     * @return {@link PinotWriterSegment} accepting at least one more element
+     */
+    private PinotWriterSegment<IN> getOrCreateInProgressSegment() {
+        final PinotWriterSegment<IN> latestSegment = Iterables.getLast(activeSegments, null);
+        if (latestSegment == null || !latestSegment.acceptsElements()) {
+            final PinotWriterSegment<IN> inProgressSegment = new PinotWriterSegment<>(maxRowsPerSegment, jsonSerializer, fsAdapter);
+            activeSegments.add(inProgressSegment);
+            return inProgressSegment;
+        }
+        return latestSegment;
+    }
+
+    /**
+     * As we do not need to save any information in snapshots,
+     * this method always returns an empty ArrayList.
+     *
+     * @return always an empty ArrayList
+     */
+    @Override
+    public List<Void> snapshotState() {

Review comment:
       You are totally right, many thanks for pointing that out! I've included a `PinotWriterState` that snapshots the necessary information (elements, min&max timestamps) from the `PinotWriterSegment` which is currently in-progress.  
   Upon `PinotSinkWriter` creation the state would then get recovered and transformed into a new in-progress `PinotWriterSegment`.




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

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