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/12 20:57:47 UTC

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

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