liuml07 commented on code in PR #135:
URL: 
https://github.com/apache/flink-connector-elasticsearch/pull/135#discussion_r2715586764


##########
flink-connector-elasticsearch8/src/main/java/org/apache/flink/connector/elasticsearch/table/AbstractTimeIndexGenerator.java:
##########
@@ -0,0 +1,23 @@
+package org.apache.flink.connector.elasticsearch.table;

Review Comment:
   We should add Apache License for new files. Same as other new files.



##########
flink-connector-elasticsearch8/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticSearch8AsyncDynamicSink.java:
##########
@@ -0,0 +1,214 @@
+package org.apache.flink.connector.elasticsearch.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSink;
+import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSinkBuilder;
+import 
org.apache.flink.connector.elasticsearch.sink.Elasticsearch8AsyncSinkBuilder;
+import org.apache.flink.connector.elasticsearch.sink.Operation;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.StringUtils;
+
+import org.apache.http.HttpHost;
+
+import java.time.ZoneId;
+import java.util.List;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** ElasticSearch backed {@link AsyncDynamicTableSink}. */
+@Internal
+public class ElasticSearch8AsyncDynamicSink extends 
AsyncDynamicTableSink<Operation> {
+    final transient EncodingFormat<SerializationSchema<RowData>> format;
+    final DataType physicalRowDataType;
+    final List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex;
+    final Elasticsearch8Configuration config;
+    final ZoneId localTimeZoneId;
+
+    final String summaryString;
+    final boolean isDynamicIndexWithSystemTime;
+
+    public ElasticSearch8AsyncDynamicSink(
+            EncodingFormat<SerializationSchema<RowData>> format,
+            Elasticsearch8Configuration config,
+            List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex,
+            DataType physicalRowDataType,
+            String summaryString,
+            ZoneId localTimeZoneId) {
+        super(
+                config.getBulkFlushMaxActions(),
+                config.getBulkFlushMaxInFlightActions(),
+                config.getBulkFlushMaxBufferedActions(),
+                config.getBulkFlushMaxByteSize().getBytes(),
+                config.getBulkFlushInterval());
+        this.format = checkNotNull(format);
+        this.physicalRowDataType = checkNotNull(physicalRowDataType);
+        this.primaryKeyLogicalTypesWithIndex = 
checkNotNull(primaryKeyLogicalTypesWithIndex);
+        this.config = checkNotNull(config);
+        this.summaryString = checkNotNull(summaryString);
+        this.localTimeZoneId = localTimeZoneId;
+        this.isDynamicIndexWithSystemTime = isDynamicIndexWithSystemTime();
+    }
+
+    public boolean isDynamicIndexWithSystemTime() {
+        IndexGeneratorFactory.IndexHelper indexHelper = new 
IndexGeneratorFactory.IndexHelper();
+        return 
indexHelper.checkIsDynamicIndexWithSystemTimeFormat(config.getIndex());
+    }
+
+    Function<RowData, String> createKeyExtractor() {
+        return KeyExtractor.createKeyExtractor(
+                primaryKeyLogicalTypesWithIndex, config.getKeyDelimiter());
+    }
+
+    IndexGenerator createIndexGenerator() {
+        return IndexGeneratorFactory.createIndexGenerator(
+                config.getIndex(),
+                DataType.getFieldNames(physicalRowDataType),
+                DataType.getFieldDataTypes(physicalRowDataType),
+                localTimeZoneId);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        ChangelogMode.Builder builder = ChangelogMode.newBuilder();
+        for (RowKind kind : requestedMode.getContainedKinds()) {
+            if (kind != RowKind.UPDATE_BEFORE) {
+                builder.addContainedKind(kind);
+            }
+        }
+        if (isDynamicIndexWithSystemTime && 
!requestedMode.containsOnly(RowKind.INSERT)) {
+            throw new ValidationException(
+                    "Dynamic indexing based on system time only works on 
append only stream.");
+        }
+        return builder.build();
+    }
+
+    @Override
+    public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
+        Elasticsearch8AsyncSinkBuilder<RowData> builder = new 
Elasticsearch8AsyncSinkBuilder<>();
+
+        builder.setHosts(config.getHosts().toArray(new HttpHost[0]));
+        builder.setMaxBatchSize(config.getBulkFlushMaxActions());
+        
builder.setMaxBufferedRequests(config.getBulkFlushMaxBufferedActions());
+        
builder.setMaxBatchSizeInBytes(config.getBulkFlushMaxByteSize().getBytes());
+        builder.setMaxTimeInBufferMS(config.getBulkFlushInterval());
+        builder.setElementConverter(
+                new RowDataElementConverter(
+                        physicalRowDataType, createIndexGenerator(), 
createKeyExtractor()));
+
+        if (config.getUsername().isPresent()
+                && 
!StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) {
+            builder.setUsername(config.getUsername().get());
+        }
+
+        if (config.getPassword().isPresent()
+                && 
!StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) {
+            builder.setPassword(config.getPassword().get());
+        }
+
+        if (config.getPathPrefix().isPresent()
+                && 
!StringUtils.isNullOrWhitespaceOnly(config.getPathPrefix().get())) {
+            builder.setConnectionPathPrefix(config.getPathPrefix().get());
+        }
+
+        if (config.getConnectionRequestTimeout().isPresent()) {
+            builder.setConnectionRequestTimeout(
+                    (int) 
config.getConnectionRequestTimeout().get().getSeconds());
+        }
+
+        if (config.getConnectionTimeout().isPresent()) {
+            builder.setConnectionTimeout((int) 
config.getConnectionTimeout().get().getSeconds());

Review Comment:
   same as above



##########
flink-connector-elasticsearch8/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticSearch8AsyncDynamicSink.java:
##########
@@ -0,0 +1,214 @@
+package org.apache.flink.connector.elasticsearch.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSink;
+import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSinkBuilder;
+import 
org.apache.flink.connector.elasticsearch.sink.Elasticsearch8AsyncSinkBuilder;
+import org.apache.flink.connector.elasticsearch.sink.Operation;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.StringUtils;
+
+import org.apache.http.HttpHost;
+
+import java.time.ZoneId;
+import java.util.List;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** ElasticSearch backed {@link AsyncDynamicTableSink}. */
+@Internal
+public class ElasticSearch8AsyncDynamicSink extends 
AsyncDynamicTableSink<Operation> {

Review Comment:
   As the base class does, should we override the `equals()` and `hashCode()` 
methods? I remember vaguely Flink's planner uses `equals()` for sink 
deduplication / optimization.



##########
flink-connector-elasticsearch8/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java:
##########
@@ -0,0 +1,298 @@
+package org.apache.flink.connector.elasticsearch.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import javax.annotation.Nonnull;
+
+import java.io.Serializable;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Factory of {@link IndexGenerator}.
+ *
+ * <p>Flink supports both static index and dynamic index.
+ *
+ * <p>If you want to have a static index, this option value should be a plain 
string, e.g.
+ * 'myusers', all the records will be consistently written into "myusers" 
index.
+ *
+ * <p>If you want to have a dynamic index, you can use '{field_name}' to 
reference a field value in
+ * the record to dynamically generate a target index. You can also use
+ * '{field_name|date_format_string}' to convert a field value of 
TIMESTAMP/DATE/TIME type into the
+ * format specified by date_format_string. The date_format_string is 
compatible with {@link
+ * java.text.SimpleDateFormat}. For example, if the option value is 
'myusers_{log_ts|yyyy-MM-dd}',
+ * then a record with log_ts field value 2020-03-27 12:25:55 will be written 
into
+ * "myusers_2020-03-27" index.
+ */
+@Internal
+final class IndexGeneratorFactory {
+
+    private IndexGeneratorFactory() {}
+
+    public static IndexGenerator createIndexGenerator(
+            String index,
+            List<String> fieldNames,
+            List<DataType> dataTypes,
+            ZoneId localTimeZoneId) {
+        final IndexHelper indexHelper = new IndexHelper();
+        if (indexHelper.checkIsDynamicIndex(index)) {
+            return createRuntimeIndexGenerator(
+                    index,
+                    fieldNames.toArray(new String[0]),
+                    dataTypes.toArray(new DataType[0]),
+                    indexHelper,
+                    localTimeZoneId);
+        } else {
+            return new StaticIndexGenerator(index);
+        }
+    }
+
+    public static IndexGenerator createIndexGenerator(
+            String index, List<String> fieldNames, List<DataType> dataTypes) {
+        return createIndexGenerator(index, fieldNames, dataTypes, 
ZoneId.systemDefault());
+    }
+
+    interface DynamicFormatter extends Serializable {
+        String format(@Nonnull Object fieldValue, DateTimeFormatter formatter);
+    }
+
+    private static IndexGenerator createRuntimeIndexGenerator(
+            String index,
+            String[] fieldNames,
+            DataType[] fieldTypes,
+            IndexHelper indexHelper,
+            ZoneId localTimeZoneId) {
+        final String dynamicIndexPatternStr = 
indexHelper.extractDynamicIndexPatternStr(index);
+        final String indexPrefix = index.substring(0, 
index.indexOf(dynamicIndexPatternStr));
+        final String indexSuffix =
+                index.substring(indexPrefix.length() + 
dynamicIndexPatternStr.length());
+
+        if (indexHelper.checkIsDynamicIndexWithSystemTimeFormat(index)) {
+            final String dateTimeFormat =
+                    indexHelper.extractDateFormat(
+                            index, 
LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
+            return new AbstractTimeIndexGenerator(index, dateTimeFormat) {
+                @Override
+                public String generate(RowData row) {
+                    return indexPrefix
+                            
.concat(LocalDateTime.now(localTimeZoneId).format(dateTimeFormatter))

Review Comment:
   wondering if we should use `StringBuilder` for a bit performance win given 
this is hot path



##########
flink-connector-elasticsearch8/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticSearch8AsyncDynamicSink.java:
##########
@@ -0,0 +1,214 @@
+package org.apache.flink.connector.elasticsearch.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSink;
+import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSinkBuilder;
+import 
org.apache.flink.connector.elasticsearch.sink.Elasticsearch8AsyncSinkBuilder;
+import org.apache.flink.connector.elasticsearch.sink.Operation;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.StringUtils;
+
+import org.apache.http.HttpHost;
+
+import java.time.ZoneId;
+import java.util.List;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** ElasticSearch backed {@link AsyncDynamicTableSink}. */
+@Internal
+public class ElasticSearch8AsyncDynamicSink extends 
AsyncDynamicTableSink<Operation> {
+    final transient EncodingFormat<SerializationSchema<RowData>> format;
+    final DataType physicalRowDataType;
+    final List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex;
+    final Elasticsearch8Configuration config;
+    final ZoneId localTimeZoneId;
+
+    final String summaryString;
+    final boolean isDynamicIndexWithSystemTime;
+
+    public ElasticSearch8AsyncDynamicSink(
+            EncodingFormat<SerializationSchema<RowData>> format,
+            Elasticsearch8Configuration config,
+            List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex,
+            DataType physicalRowDataType,
+            String summaryString,
+            ZoneId localTimeZoneId) {
+        super(
+                config.getBulkFlushMaxActions(),
+                config.getBulkFlushMaxInFlightActions(),
+                config.getBulkFlushMaxBufferedActions(),
+                config.getBulkFlushMaxByteSize().getBytes(),
+                config.getBulkFlushInterval());
+        this.format = checkNotNull(format);
+        this.physicalRowDataType = checkNotNull(physicalRowDataType);
+        this.primaryKeyLogicalTypesWithIndex = 
checkNotNull(primaryKeyLogicalTypesWithIndex);
+        this.config = checkNotNull(config);
+        this.summaryString = checkNotNull(summaryString);
+        this.localTimeZoneId = localTimeZoneId;
+        this.isDynamicIndexWithSystemTime = isDynamicIndexWithSystemTime();
+    }
+
+    public boolean isDynamicIndexWithSystemTime() {
+        IndexGeneratorFactory.IndexHelper indexHelper = new 
IndexGeneratorFactory.IndexHelper();
+        return 
indexHelper.checkIsDynamicIndexWithSystemTimeFormat(config.getIndex());
+    }
+
+    Function<RowData, String> createKeyExtractor() {
+        return KeyExtractor.createKeyExtractor(
+                primaryKeyLogicalTypesWithIndex, config.getKeyDelimiter());
+    }
+
+    IndexGenerator createIndexGenerator() {
+        return IndexGeneratorFactory.createIndexGenerator(
+                config.getIndex(),
+                DataType.getFieldNames(physicalRowDataType),
+                DataType.getFieldDataTypes(physicalRowDataType),
+                localTimeZoneId);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        ChangelogMode.Builder builder = ChangelogMode.newBuilder();
+        for (RowKind kind : requestedMode.getContainedKinds()) {
+            if (kind != RowKind.UPDATE_BEFORE) {
+                builder.addContainedKind(kind);
+            }
+        }
+        if (isDynamicIndexWithSystemTime && 
!requestedMode.containsOnly(RowKind.INSERT)) {
+            throw new ValidationException(
+                    "Dynamic indexing based on system time only works on 
append only stream.");
+        }
+        return builder.build();
+    }
+
+    @Override
+    public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
+        Elasticsearch8AsyncSinkBuilder<RowData> builder = new 
Elasticsearch8AsyncSinkBuilder<>();
+
+        builder.setHosts(config.getHosts().toArray(new HttpHost[0]));
+        builder.setMaxBatchSize(config.getBulkFlushMaxActions());
+        
builder.setMaxBufferedRequests(config.getBulkFlushMaxBufferedActions());
+        
builder.setMaxBatchSizeInBytes(config.getBulkFlushMaxByteSize().getBytes());
+        builder.setMaxTimeInBufferMS(config.getBulkFlushInterval());
+        builder.setElementConverter(
+                new RowDataElementConverter(
+                        physicalRowDataType, createIndexGenerator(), 
createKeyExtractor()));
+
+        if (config.getUsername().isPresent()
+                && 
!StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) {
+            builder.setUsername(config.getUsername().get());
+        }

Review Comment:
   same as all other if-isPresent-then-get cases



##########
flink-connector-elasticsearch8/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticSearch8AsyncDynamicSink.java:
##########
@@ -0,0 +1,214 @@
+package org.apache.flink.connector.elasticsearch.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSink;
+import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSinkBuilder;
+import 
org.apache.flink.connector.elasticsearch.sink.Elasticsearch8AsyncSinkBuilder;
+import org.apache.flink.connector.elasticsearch.sink.Operation;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.StringUtils;
+
+import org.apache.http.HttpHost;
+
+import java.time.ZoneId;
+import java.util.List;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** ElasticSearch backed {@link AsyncDynamicTableSink}. */
+@Internal
+public class ElasticSearch8AsyncDynamicSink extends 
AsyncDynamicTableSink<Operation> {
+    final transient EncodingFormat<SerializationSchema<RowData>> format;
+    final DataType physicalRowDataType;
+    final List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex;
+    final Elasticsearch8Configuration config;
+    final ZoneId localTimeZoneId;
+
+    final String summaryString;
+    final boolean isDynamicIndexWithSystemTime;
+
+    public ElasticSearch8AsyncDynamicSink(
+            EncodingFormat<SerializationSchema<RowData>> format,
+            Elasticsearch8Configuration config,
+            List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex,
+            DataType physicalRowDataType,
+            String summaryString,
+            ZoneId localTimeZoneId) {
+        super(
+                config.getBulkFlushMaxActions(),
+                config.getBulkFlushMaxInFlightActions(),
+                config.getBulkFlushMaxBufferedActions(),
+                config.getBulkFlushMaxByteSize().getBytes(),
+                config.getBulkFlushInterval());
+        this.format = checkNotNull(format);
+        this.physicalRowDataType = checkNotNull(physicalRowDataType);
+        this.primaryKeyLogicalTypesWithIndex = 
checkNotNull(primaryKeyLogicalTypesWithIndex);
+        this.config = checkNotNull(config);
+        this.summaryString = checkNotNull(summaryString);
+        this.localTimeZoneId = localTimeZoneId;
+        this.isDynamicIndexWithSystemTime = isDynamicIndexWithSystemTime();
+    }
+
+    public boolean isDynamicIndexWithSystemTime() {
+        IndexGeneratorFactory.IndexHelper indexHelper = new 
IndexGeneratorFactory.IndexHelper();
+        return 
indexHelper.checkIsDynamicIndexWithSystemTimeFormat(config.getIndex());
+    }
+
+    Function<RowData, String> createKeyExtractor() {
+        return KeyExtractor.createKeyExtractor(
+                primaryKeyLogicalTypesWithIndex, config.getKeyDelimiter());
+    }
+
+    IndexGenerator createIndexGenerator() {
+        return IndexGeneratorFactory.createIndexGenerator(
+                config.getIndex(),
+                DataType.getFieldNames(physicalRowDataType),
+                DataType.getFieldDataTypes(physicalRowDataType),
+                localTimeZoneId);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        ChangelogMode.Builder builder = ChangelogMode.newBuilder();
+        for (RowKind kind : requestedMode.getContainedKinds()) {
+            if (kind != RowKind.UPDATE_BEFORE) {
+                builder.addContainedKind(kind);
+            }
+        }
+        if (isDynamicIndexWithSystemTime && 
!requestedMode.containsOnly(RowKind.INSERT)) {
+            throw new ValidationException(
+                    "Dynamic indexing based on system time only works on 
append only stream.");
+        }
+        return builder.build();
+    }
+
+    @Override
+    public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
+        Elasticsearch8AsyncSinkBuilder<RowData> builder = new 
Elasticsearch8AsyncSinkBuilder<>();
+
+        builder.setHosts(config.getHosts().toArray(new HttpHost[0]));
+        builder.setMaxBatchSize(config.getBulkFlushMaxActions());
+        
builder.setMaxBufferedRequests(config.getBulkFlushMaxBufferedActions());
+        
builder.setMaxBatchSizeInBytes(config.getBulkFlushMaxByteSize().getBytes());
+        builder.setMaxTimeInBufferMS(config.getBulkFlushInterval());
+        builder.setElementConverter(
+                new RowDataElementConverter(
+                        physicalRowDataType, createIndexGenerator(), 
createKeyExtractor()));
+
+        if (config.getUsername().isPresent()
+                && 
!StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) {
+            builder.setUsername(config.getUsername().get());
+        }

Review Comment:
   Not sure if fluent approach is clearer to you?
   ```java
     config.getUsername()                                                       
                                                                                
                                                                                
                                 
             .filter(username -> !StringUtils.isNullOrWhitespaceOnly(username)) 
                                                                                
                                                                                
                                 
             .ifPresent(builder::setUsername);  
   ```



##########
flink-connector-elasticsearch8/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java:
##########
@@ -0,0 +1,298 @@
+package org.apache.flink.connector.elasticsearch.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import javax.annotation.Nonnull;
+
+import java.io.Serializable;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Factory of {@link IndexGenerator}.
+ *
+ * <p>Flink supports both static index and dynamic index.
+ *
+ * <p>If you want to have a static index, this option value should be a plain 
string, e.g.
+ * 'myusers', all the records will be consistently written into "myusers" 
index.
+ *
+ * <p>If you want to have a dynamic index, you can use '{field_name}' to 
reference a field value in
+ * the record to dynamically generate a target index. You can also use
+ * '{field_name|date_format_string}' to convert a field value of 
TIMESTAMP/DATE/TIME type into the
+ * format specified by date_format_string. The date_format_string is 
compatible with {@link
+ * java.text.SimpleDateFormat}. For example, if the option value is 
'myusers_{log_ts|yyyy-MM-dd}',
+ * then a record with log_ts field value 2020-03-27 12:25:55 will be written 
into
+ * "myusers_2020-03-27" index.
+ */
+@Internal
+final class IndexGeneratorFactory {
+
+    private IndexGeneratorFactory() {}
+
+    public static IndexGenerator createIndexGenerator(
+            String index,
+            List<String> fieldNames,
+            List<DataType> dataTypes,
+            ZoneId localTimeZoneId) {
+        final IndexHelper indexHelper = new IndexHelper();
+        if (indexHelper.checkIsDynamicIndex(index)) {
+            return createRuntimeIndexGenerator(
+                    index,
+                    fieldNames.toArray(new String[0]),
+                    dataTypes.toArray(new DataType[0]),
+                    indexHelper,
+                    localTimeZoneId);
+        } else {
+            return new StaticIndexGenerator(index);
+        }
+    }
+
+    public static IndexGenerator createIndexGenerator(
+            String index, List<String> fieldNames, List<DataType> dataTypes) {
+        return createIndexGenerator(index, fieldNames, dataTypes, 
ZoneId.systemDefault());
+    }
+
+    interface DynamicFormatter extends Serializable {
+        String format(@Nonnull Object fieldValue, DateTimeFormatter formatter);
+    }
+
+    private static IndexGenerator createRuntimeIndexGenerator(
+            String index,
+            String[] fieldNames,
+            DataType[] fieldTypes,
+            IndexHelper indexHelper,
+            ZoneId localTimeZoneId) {
+        final String dynamicIndexPatternStr = 
indexHelper.extractDynamicIndexPatternStr(index);
+        final String indexPrefix = index.substring(0, 
index.indexOf(dynamicIndexPatternStr));
+        final String indexSuffix =
+                index.substring(indexPrefix.length() + 
dynamicIndexPatternStr.length());
+
+        if (indexHelper.checkIsDynamicIndexWithSystemTimeFormat(index)) {
+            final String dateTimeFormat =
+                    indexHelper.extractDateFormat(
+                            index, 
LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
+            return new AbstractTimeIndexGenerator(index, dateTimeFormat) {
+                @Override
+                public String generate(RowData row) {
+                    return indexPrefix
+                            
.concat(LocalDateTime.now(localTimeZoneId).format(dateTimeFormatter))
+                            .concat(indexSuffix);
+                }
+            };
+        }
+
+        final boolean isDynamicIndexWithFormat = 
indexHelper.checkIsDynamicIndexWithFormat(index);
+        final int indexFieldPos =
+                indexHelper.extractIndexFieldPos(index, fieldNames, 
isDynamicIndexWithFormat);
+        final LogicalType indexFieldType = 
fieldTypes[indexFieldPos].getLogicalType();
+        final LogicalTypeRoot indexFieldLogicalTypeRoot = 
indexFieldType.getTypeRoot();
+
+        // validate index field type
+        indexHelper.validateIndexFieldType(indexFieldLogicalTypeRoot);
+
+        // time extract dynamic index pattern
+        final RowData.FieldGetter fieldGetter =
+                RowData.createFieldGetter(indexFieldType, indexFieldPos);
+
+        if (isDynamicIndexWithFormat) {
+            final String dateTimeFormat =
+                    indexHelper.extractDateFormat(index, 
indexFieldLogicalTypeRoot);
+            DynamicFormatter formatFunction =
+                    createFormatFunction(
+                            indexFieldType, indexFieldLogicalTypeRoot, 
localTimeZoneId);
+
+            return new AbstractTimeIndexGenerator(index, dateTimeFormat) {
+                @Override
+                public String generate(RowData row) {
+                    Object fieldOrNull = fieldGetter.getFieldOrNull(row);
+                    final String formattedField;
+                    if (fieldOrNull != null) {
+                        formattedField = formatFunction.format(fieldOrNull, 
dateTimeFormatter);
+                    } else {
+                        formattedField = "null";
+                    }
+                    return 
indexPrefix.concat(formattedField).concat(indexSuffix);
+                }
+            };
+        }
+        // general dynamic index pattern
+        return new IndexGeneratorBase(index) {
+            @Override
+            public String generate(RowData row) {
+                Object indexField = fieldGetter.getFieldOrNull(row);
+                return indexPrefix
+                        .concat(indexField == null ? "null" : 
indexField.toString())
+                        .concat(indexSuffix);
+            }
+        };
+    }
+
+    private static DynamicFormatter createFormatFunction(
+            LogicalType indexFieldType,
+            LogicalTypeRoot indexFieldLogicalTypeRoot,
+            ZoneId localTimeZoneId) {
+        switch (indexFieldLogicalTypeRoot) {
+            case DATE:
+                return (value, dateTimeFormatter) -> {
+                    Integer indexField = (Integer) value;
+                    return 
LocalDate.ofEpochDay(indexField).format(dateTimeFormatter);
+                };
+            case TIME_WITHOUT_TIME_ZONE:
+                return (value, dateTimeFormatter) -> {
+                    Integer indexField = (Integer) value;
+                    return LocalTime.ofNanoOfDay(indexField * 
1_000_000L).format(dateTimeFormatter);
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return (value, dateTimeFormatter) -> {
+                    TimestampData indexField = (TimestampData) value;
+                    return 
indexField.toLocalDateTime().format(dateTimeFormatter);
+                };
+            case TIMESTAMP_WITH_TIME_ZONE:
+                throw new UnsupportedOperationException(
+                        "TIMESTAMP_WITH_TIME_ZONE is not supported yet");
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return (value, dateTimeFormatter) -> {
+                    TimestampData indexField = (TimestampData) value;
+                    return 
indexField.toInstant().atZone(localTimeZoneId).format(dateTimeFormatter);
+                };
+            default:
+                throw new TableException(
+                        String.format(
+                                "Unsupported type '%s' found in Elasticsearch 
dynamic index field, "
+                                        + "time-related pattern only support 
types are: DATE,TIME,TIMESTAMP.",
+                                indexFieldType));
+        }
+    }
+
+    /**
+     * Helper class for {@link IndexGeneratorFactory}, this helper can use to 
validate index field
+     * type ans parse index format from pattern.
+     */
+    static class IndexHelper {
+        private static final Pattern dynamicIndexPattern = 
Pattern.compile("\\{[^\\{\\}]+\\}?");
+        private static final Pattern dynamicIndexTimeExtractPattern =
+                Pattern.compile(".*\\{.+\\|.*\\}.*");
+        private static final Pattern dynamicIndexSystemTimeExtractPattern =
+                Pattern.compile(
+                        
".*\\{\\s*(now\\(\\s*\\)|NOW\\(\\s*\\)|current_timestamp|CURRENT_TIMESTAMP)\\s*\\|.*\\}.*");
+        private static final List<LogicalTypeRoot> supportedTypes = new 
ArrayList<>();
+        private static final Map<LogicalTypeRoot, String> defaultFormats = new 
HashMap<>();
+
+        static {
+            // time related types
+            supportedTypes.add(LogicalTypeRoot.DATE);
+            supportedTypes.add(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE);
+            supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE);
+            supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE);

Review Comment:
   not really supported as above in line 166?
   ```
               case TIMESTAMP_WITH_TIME_ZONE:
                   throw new UnsupportedOperationException(
                           "TIMESTAMP_WITH_TIME_ZONE is not supported yet");
   ```
   Shall we remove it here?



##########
flink-connector-elasticsearch8/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticSearch8AsyncDynamicSink.java:
##########
@@ -0,0 +1,214 @@
+package org.apache.flink.connector.elasticsearch.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSink;
+import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSinkBuilder;
+import 
org.apache.flink.connector.elasticsearch.sink.Elasticsearch8AsyncSinkBuilder;
+import org.apache.flink.connector.elasticsearch.sink.Operation;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.StringUtils;
+
+import org.apache.http.HttpHost;
+
+import java.time.ZoneId;
+import java.util.List;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** ElasticSearch backed {@link AsyncDynamicTableSink}. */
+@Internal
+public class ElasticSearch8AsyncDynamicSink extends 
AsyncDynamicTableSink<Operation> {
+    final transient EncodingFormat<SerializationSchema<RowData>> format;
+    final DataType physicalRowDataType;
+    final List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex;
+    final Elasticsearch8Configuration config;
+    final ZoneId localTimeZoneId;
+
+    final String summaryString;
+    final boolean isDynamicIndexWithSystemTime;
+
+    public ElasticSearch8AsyncDynamicSink(
+            EncodingFormat<SerializationSchema<RowData>> format,
+            Elasticsearch8Configuration config,
+            List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex,
+            DataType physicalRowDataType,
+            String summaryString,
+            ZoneId localTimeZoneId) {
+        super(
+                config.getBulkFlushMaxActions(),
+                config.getBulkFlushMaxInFlightActions(),
+                config.getBulkFlushMaxBufferedActions(),
+                config.getBulkFlushMaxByteSize().getBytes(),
+                config.getBulkFlushInterval());
+        this.format = checkNotNull(format);
+        this.physicalRowDataType = checkNotNull(physicalRowDataType);
+        this.primaryKeyLogicalTypesWithIndex = 
checkNotNull(primaryKeyLogicalTypesWithIndex);
+        this.config = checkNotNull(config);
+        this.summaryString = checkNotNull(summaryString);
+        this.localTimeZoneId = localTimeZoneId;
+        this.isDynamicIndexWithSystemTime = isDynamicIndexWithSystemTime();
+    }
+
+    public boolean isDynamicIndexWithSystemTime() {
+        IndexGeneratorFactory.IndexHelper indexHelper = new 
IndexGeneratorFactory.IndexHelper();
+        return 
indexHelper.checkIsDynamicIndexWithSystemTimeFormat(config.getIndex());
+    }
+
+    Function<RowData, String> createKeyExtractor() {
+        return KeyExtractor.createKeyExtractor(
+                primaryKeyLogicalTypesWithIndex, config.getKeyDelimiter());
+    }
+
+    IndexGenerator createIndexGenerator() {
+        return IndexGeneratorFactory.createIndexGenerator(
+                config.getIndex(),
+                DataType.getFieldNames(physicalRowDataType),
+                DataType.getFieldDataTypes(physicalRowDataType),
+                localTimeZoneId);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        ChangelogMode.Builder builder = ChangelogMode.newBuilder();
+        for (RowKind kind : requestedMode.getContainedKinds()) {
+            if (kind != RowKind.UPDATE_BEFORE) {
+                builder.addContainedKind(kind);
+            }
+        }
+        if (isDynamicIndexWithSystemTime && 
!requestedMode.containsOnly(RowKind.INSERT)) {
+            throw new ValidationException(
+                    "Dynamic indexing based on system time only works on 
append only stream.");
+        }
+        return builder.build();
+    }
+
+    @Override
+    public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
+        Elasticsearch8AsyncSinkBuilder<RowData> builder = new 
Elasticsearch8AsyncSinkBuilder<>();
+
+        builder.setHosts(config.getHosts().toArray(new HttpHost[0]));
+        builder.setMaxBatchSize(config.getBulkFlushMaxActions());
+        
builder.setMaxBufferedRequests(config.getBulkFlushMaxBufferedActions());
+        
builder.setMaxBatchSizeInBytes(config.getBulkFlushMaxByteSize().getBytes());
+        builder.setMaxTimeInBufferMS(config.getBulkFlushInterval());
+        builder.setElementConverter(
+                new RowDataElementConverter(
+                        physicalRowDataType, createIndexGenerator(), 
createKeyExtractor()));
+
+        if (config.getUsername().isPresent()
+                && 
!StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) {
+            builder.setUsername(config.getUsername().get());
+        }
+
+        if (config.getPassword().isPresent()
+                && 
!StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) {
+            builder.setPassword(config.getPassword().get());
+        }
+
+        if (config.getPathPrefix().isPresent()
+                && 
!StringUtils.isNullOrWhitespaceOnly(config.getPathPrefix().get())) {
+            builder.setConnectionPathPrefix(config.getPathPrefix().get());
+        }
+
+        if (config.getConnectionRequestTimeout().isPresent()) {
+            builder.setConnectionRequestTimeout(
+                    (int) 
config.getConnectionRequestTimeout().get().getSeconds());
+        }

Review Comment:
   Shall this be milliseconds not seconds?



##########
flink-connector-elasticsearch8/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticSearch8AsyncDynamicSink.java:
##########
@@ -0,0 +1,214 @@
+package org.apache.flink.connector.elasticsearch.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSink;
+import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSinkBuilder;
+import 
org.apache.flink.connector.elasticsearch.sink.Elasticsearch8AsyncSinkBuilder;
+import org.apache.flink.connector.elasticsearch.sink.Operation;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.StringUtils;
+
+import org.apache.http.HttpHost;
+
+import java.time.ZoneId;
+import java.util.List;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** ElasticSearch backed {@link AsyncDynamicTableSink}. */
+@Internal
+public class ElasticSearch8AsyncDynamicSink extends 
AsyncDynamicTableSink<Operation> {
+    final transient EncodingFormat<SerializationSchema<RowData>> format;
+    final DataType physicalRowDataType;
+    final List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex;
+    final Elasticsearch8Configuration config;
+    final ZoneId localTimeZoneId;
+
+    final String summaryString;
+    final boolean isDynamicIndexWithSystemTime;
+
+    public ElasticSearch8AsyncDynamicSink(
+            EncodingFormat<SerializationSchema<RowData>> format,
+            Elasticsearch8Configuration config,
+            List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex,
+            DataType physicalRowDataType,
+            String summaryString,
+            ZoneId localTimeZoneId) {
+        super(
+                config.getBulkFlushMaxActions(),
+                config.getBulkFlushMaxInFlightActions(),
+                config.getBulkFlushMaxBufferedActions(),
+                config.getBulkFlushMaxByteSize().getBytes(),
+                config.getBulkFlushInterval());
+        this.format = checkNotNull(format);
+        this.physicalRowDataType = checkNotNull(physicalRowDataType);
+        this.primaryKeyLogicalTypesWithIndex = 
checkNotNull(primaryKeyLogicalTypesWithIndex);
+        this.config = checkNotNull(config);
+        this.summaryString = checkNotNull(summaryString);
+        this.localTimeZoneId = localTimeZoneId;
+        this.isDynamicIndexWithSystemTime = isDynamicIndexWithSystemTime();
+    }
+
+    public boolean isDynamicIndexWithSystemTime() {
+        IndexGeneratorFactory.IndexHelper indexHelper = new 
IndexGeneratorFactory.IndexHelper();
+        return 
indexHelper.checkIsDynamicIndexWithSystemTimeFormat(config.getIndex());
+    }
+
+    Function<RowData, String> createKeyExtractor() {
+        return KeyExtractor.createKeyExtractor(
+                primaryKeyLogicalTypesWithIndex, config.getKeyDelimiter());
+    }
+
+    IndexGenerator createIndexGenerator() {
+        return IndexGeneratorFactory.createIndexGenerator(
+                config.getIndex(),
+                DataType.getFieldNames(physicalRowDataType),
+                DataType.getFieldDataTypes(physicalRowDataType),
+                localTimeZoneId);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        ChangelogMode.Builder builder = ChangelogMode.newBuilder();
+        for (RowKind kind : requestedMode.getContainedKinds()) {
+            if (kind != RowKind.UPDATE_BEFORE) {
+                builder.addContainedKind(kind);
+            }
+        }
+        if (isDynamicIndexWithSystemTime && 
!requestedMode.containsOnly(RowKind.INSERT)) {
+            throw new ValidationException(

Review Comment:
   This makes sense. But does Elasticsearch sink requires a primary key to be 
defined on the table when performing UPDATE or DELETE operations? If so we need 
to validate that as well.



##########
flink-connector-elasticsearch8/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java:
##########
@@ -0,0 +1,298 @@
+package org.apache.flink.connector.elasticsearch.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import javax.annotation.Nonnull;
+
+import java.io.Serializable;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Factory of {@link IndexGenerator}.
+ *
+ * <p>Flink supports both static index and dynamic index.
+ *
+ * <p>If you want to have a static index, this option value should be a plain 
string, e.g.
+ * 'myusers', all the records will be consistently written into "myusers" 
index.
+ *
+ * <p>If you want to have a dynamic index, you can use '{field_name}' to 
reference a field value in
+ * the record to dynamically generate a target index. You can also use
+ * '{field_name|date_format_string}' to convert a field value of 
TIMESTAMP/DATE/TIME type into the
+ * format specified by date_format_string. The date_format_string is 
compatible with {@link
+ * java.text.SimpleDateFormat}. For example, if the option value is 
'myusers_{log_ts|yyyy-MM-dd}',
+ * then a record with log_ts field value 2020-03-27 12:25:55 will be written 
into
+ * "myusers_2020-03-27" index.
+ */
+@Internal
+final class IndexGeneratorFactory {
+
+    private IndexGeneratorFactory() {}
+
+    public static IndexGenerator createIndexGenerator(
+            String index,
+            List<String> fieldNames,
+            List<DataType> dataTypes,
+            ZoneId localTimeZoneId) {
+        final IndexHelper indexHelper = new IndexHelper();
+        if (indexHelper.checkIsDynamicIndex(index)) {
+            return createRuntimeIndexGenerator(
+                    index,
+                    fieldNames.toArray(new String[0]),
+                    dataTypes.toArray(new DataType[0]),
+                    indexHelper,
+                    localTimeZoneId);
+        } else {
+            return new StaticIndexGenerator(index);
+        }
+    }
+
+    public static IndexGenerator createIndexGenerator(
+            String index, List<String> fieldNames, List<DataType> dataTypes) {
+        return createIndexGenerator(index, fieldNames, dataTypes, 
ZoneId.systemDefault());
+    }
+
+    interface DynamicFormatter extends Serializable {
+        String format(@Nonnull Object fieldValue, DateTimeFormatter formatter);
+    }
+
+    private static IndexGenerator createRuntimeIndexGenerator(
+            String index,
+            String[] fieldNames,
+            DataType[] fieldTypes,
+            IndexHelper indexHelper,
+            ZoneId localTimeZoneId) {
+        final String dynamicIndexPatternStr = 
indexHelper.extractDynamicIndexPatternStr(index);
+        final String indexPrefix = index.substring(0, 
index.indexOf(dynamicIndexPatternStr));
+        final String indexSuffix =
+                index.substring(indexPrefix.length() + 
dynamicIndexPatternStr.length());
+
+        if (indexHelper.checkIsDynamicIndexWithSystemTimeFormat(index)) {
+            final String dateTimeFormat =
+                    indexHelper.extractDateFormat(
+                            index, 
LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
+            return new AbstractTimeIndexGenerator(index, dateTimeFormat) {
+                @Override
+                public String generate(RowData row) {
+                    return indexPrefix
+                            
.concat(LocalDateTime.now(localTimeZoneId).format(dateTimeFormatter))

Review Comment:
   same as below



##########
flink-connector-elasticsearch8/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticSearch8AsyncDynamicSink.java:
##########
@@ -0,0 +1,214 @@
+package org.apache.flink.connector.elasticsearch.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSink;
+import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSinkBuilder;
+import 
org.apache.flink.connector.elasticsearch.sink.Elasticsearch8AsyncSinkBuilder;
+import org.apache.flink.connector.elasticsearch.sink.Operation;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.StringUtils;
+
+import org.apache.http.HttpHost;
+
+import java.time.ZoneId;
+import java.util.List;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** ElasticSearch backed {@link AsyncDynamicTableSink}. */
+@Internal
+public class ElasticSearch8AsyncDynamicSink extends 
AsyncDynamicTableSink<Operation> {
+    final transient EncodingFormat<SerializationSchema<RowData>> format;
+    final DataType physicalRowDataType;
+    final List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex;
+    final Elasticsearch8Configuration config;
+    final ZoneId localTimeZoneId;
+
+    final String summaryString;
+    final boolean isDynamicIndexWithSystemTime;
+
+    public ElasticSearch8AsyncDynamicSink(
+            EncodingFormat<SerializationSchema<RowData>> format,
+            Elasticsearch8Configuration config,
+            List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex,
+            DataType physicalRowDataType,
+            String summaryString,
+            ZoneId localTimeZoneId) {
+        super(
+                config.getBulkFlushMaxActions(),
+                config.getBulkFlushMaxInFlightActions(),
+                config.getBulkFlushMaxBufferedActions(),
+                config.getBulkFlushMaxByteSize().getBytes(),
+                config.getBulkFlushInterval());
+        this.format = checkNotNull(format);
+        this.physicalRowDataType = checkNotNull(physicalRowDataType);
+        this.primaryKeyLogicalTypesWithIndex = 
checkNotNull(primaryKeyLogicalTypesWithIndex);
+        this.config = checkNotNull(config);
+        this.summaryString = checkNotNull(summaryString);
+        this.localTimeZoneId = localTimeZoneId;
+        this.isDynamicIndexWithSystemTime = isDynamicIndexWithSystemTime();
+    }
+
+    public boolean isDynamicIndexWithSystemTime() {
+        IndexGeneratorFactory.IndexHelper indexHelper = new 
IndexGeneratorFactory.IndexHelper();
+        return 
indexHelper.checkIsDynamicIndexWithSystemTimeFormat(config.getIndex());
+    }
+
+    Function<RowData, String> createKeyExtractor() {
+        return KeyExtractor.createKeyExtractor(
+                primaryKeyLogicalTypesWithIndex, config.getKeyDelimiter());
+    }
+
+    IndexGenerator createIndexGenerator() {
+        return IndexGeneratorFactory.createIndexGenerator(
+                config.getIndex(),
+                DataType.getFieldNames(physicalRowDataType),
+                DataType.getFieldDataTypes(physicalRowDataType),
+                localTimeZoneId);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        ChangelogMode.Builder builder = ChangelogMode.newBuilder();
+        for (RowKind kind : requestedMode.getContainedKinds()) {
+            if (kind != RowKind.UPDATE_BEFORE) {
+                builder.addContainedKind(kind);
+            }
+        }
+        if (isDynamicIndexWithSystemTime && 
!requestedMode.containsOnly(RowKind.INSERT)) {
+            throw new ValidationException(

Review Comment:
   and add some tests



##########
flink-connector-elasticsearch8/pom.xml:
##########
@@ -138,6 +138,27 @@ under the License.
                        <type>test-jar</type>
                        <scope>test</scope>
                </dependency>
+
+               <dependency>
+                       <groupId>org.apache.flink</groupId>
+                       <artifactId>flink-table-runtime</artifactId>
+                       <version>${flink.version}</version>
+                       <scope>provided</scope>
+               </dependency>

Review Comment:
   shall we move this to above sections, as this section in dependencies are 
more for test?
   
   Maybe after:
   ```
                <!-- Projects depending on this project won't depend on 
flink-table-*. -->
                <dependency>
                        <groupId>org.apache.flink</groupId>
                        <artifactId>flink-table-api-java-bridge</artifactId>
                        <version>${flink.version}</version>
                        <scope>provided</scope>
                        <optional>true</optional>
                </dependency>
   ```



##########
flink-connector-elasticsearch8/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java:
##########
@@ -0,0 +1,298 @@
+package org.apache.flink.connector.elasticsearch.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import javax.annotation.Nonnull;
+
+import java.io.Serializable;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Factory of {@link IndexGenerator}.
+ *
+ * <p>Flink supports both static index and dynamic index.
+ *
+ * <p>If you want to have a static index, this option value should be a plain 
string, e.g.
+ * 'myusers', all the records will be consistently written into "myusers" 
index.
+ *
+ * <p>If you want to have a dynamic index, you can use '{field_name}' to 
reference a field value in
+ * the record to dynamically generate a target index. You can also use
+ * '{field_name|date_format_string}' to convert a field value of 
TIMESTAMP/DATE/TIME type into the
+ * format specified by date_format_string. The date_format_string is 
compatible with {@link
+ * java.text.SimpleDateFormat}. For example, if the option value is 
'myusers_{log_ts|yyyy-MM-dd}',
+ * then a record with log_ts field value 2020-03-27 12:25:55 will be written 
into
+ * "myusers_2020-03-27" index.
+ */
+@Internal
+final class IndexGeneratorFactory {
+
+    private IndexGeneratorFactory() {}
+
+    public static IndexGenerator createIndexGenerator(
+            String index,
+            List<String> fieldNames,
+            List<DataType> dataTypes,
+            ZoneId localTimeZoneId) {
+        final IndexHelper indexHelper = new IndexHelper();
+        if (indexHelper.checkIsDynamicIndex(index)) {
+            return createRuntimeIndexGenerator(
+                    index,
+                    fieldNames.toArray(new String[0]),
+                    dataTypes.toArray(new DataType[0]),
+                    indexHelper,
+                    localTimeZoneId);
+        } else {
+            return new StaticIndexGenerator(index);
+        }
+    }
+
+    public static IndexGenerator createIndexGenerator(
+            String index, List<String> fieldNames, List<DataType> dataTypes) {
+        return createIndexGenerator(index, fieldNames, dataTypes, 
ZoneId.systemDefault());
+    }
+
+    interface DynamicFormatter extends Serializable {
+        String format(@Nonnull Object fieldValue, DateTimeFormatter formatter);
+    }
+
+    private static IndexGenerator createRuntimeIndexGenerator(
+            String index,
+            String[] fieldNames,
+            DataType[] fieldTypes,
+            IndexHelper indexHelper,
+            ZoneId localTimeZoneId) {
+        final String dynamicIndexPatternStr = 
indexHelper.extractDynamicIndexPatternStr(index);
+        final String indexPrefix = index.substring(0, 
index.indexOf(dynamicIndexPatternStr));
+        final String indexSuffix =
+                index.substring(indexPrefix.length() + 
dynamicIndexPatternStr.length());
+
+        if (indexHelper.checkIsDynamicIndexWithSystemTimeFormat(index)) {
+            final String dateTimeFormat =
+                    indexHelper.extractDateFormat(
+                            index, 
LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
+            return new AbstractTimeIndexGenerator(index, dateTimeFormat) {
+                @Override
+                public String generate(RowData row) {
+                    return indexPrefix
+                            
.concat(LocalDateTime.now(localTimeZoneId).format(dateTimeFormatter))
+                            .concat(indexSuffix);
+                }
+            };
+        }
+
+        final boolean isDynamicIndexWithFormat = 
indexHelper.checkIsDynamicIndexWithFormat(index);
+        final int indexFieldPos =
+                indexHelper.extractIndexFieldPos(index, fieldNames, 
isDynamicIndexWithFormat);
+        final LogicalType indexFieldType = 
fieldTypes[indexFieldPos].getLogicalType();
+        final LogicalTypeRoot indexFieldLogicalTypeRoot = 
indexFieldType.getTypeRoot();
+
+        // validate index field type
+        indexHelper.validateIndexFieldType(indexFieldLogicalTypeRoot);
+
+        // time extract dynamic index pattern
+        final RowData.FieldGetter fieldGetter =
+                RowData.createFieldGetter(indexFieldType, indexFieldPos);
+
+        if (isDynamicIndexWithFormat) {
+            final String dateTimeFormat =
+                    indexHelper.extractDateFormat(index, 
indexFieldLogicalTypeRoot);
+            DynamicFormatter formatFunction =
+                    createFormatFunction(
+                            indexFieldType, indexFieldLogicalTypeRoot, 
localTimeZoneId);
+
+            return new AbstractTimeIndexGenerator(index, dateTimeFormat) {
+                @Override
+                public String generate(RowData row) {
+                    Object fieldOrNull = fieldGetter.getFieldOrNull(row);
+                    final String formattedField;
+                    if (fieldOrNull != null) {
+                        formattedField = formatFunction.format(fieldOrNull, 
dateTimeFormatter);
+                    } else {
+                        formattedField = "null";
+                    }
+                    return 
indexPrefix.concat(formattedField).concat(indexSuffix);
+                }
+            };
+        }
+        // general dynamic index pattern
+        return new IndexGeneratorBase(index) {
+            @Override
+            public String generate(RowData row) {
+                Object indexField = fieldGetter.getFieldOrNull(row);
+                return indexPrefix
+                        .concat(indexField == null ? "null" : 
indexField.toString())
+                        .concat(indexSuffix);
+            }
+        };
+    }
+
+    private static DynamicFormatter createFormatFunction(
+            LogicalType indexFieldType,
+            LogicalTypeRoot indexFieldLogicalTypeRoot,
+            ZoneId localTimeZoneId) {
+        switch (indexFieldLogicalTypeRoot) {
+            case DATE:
+                return (value, dateTimeFormatter) -> {
+                    Integer indexField = (Integer) value;
+                    return 
LocalDate.ofEpochDay(indexField).format(dateTimeFormatter);
+                };
+            case TIME_WITHOUT_TIME_ZONE:
+                return (value, dateTimeFormatter) -> {
+                    Integer indexField = (Integer) value;
+                    return LocalTime.ofNanoOfDay(indexField * 
1_000_000L).format(dateTimeFormatter);
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return (value, dateTimeFormatter) -> {
+                    TimestampData indexField = (TimestampData) value;
+                    return 
indexField.toLocalDateTime().format(dateTimeFormatter);
+                };
+            case TIMESTAMP_WITH_TIME_ZONE:
+                throw new UnsupportedOperationException(
+                        "TIMESTAMP_WITH_TIME_ZONE is not supported yet");
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return (value, dateTimeFormatter) -> {
+                    TimestampData indexField = (TimestampData) value;
+                    return 
indexField.toInstant().atZone(localTimeZoneId).format(dateTimeFormatter);
+                };
+            default:
+                throw new TableException(
+                        String.format(
+                                "Unsupported type '%s' found in Elasticsearch 
dynamic index field, "
+                                        + "time-related pattern only support 
types are: DATE,TIME,TIMESTAMP.",
+                                indexFieldType));
+        }
+    }
+
+    /**
+     * Helper class for {@link IndexGeneratorFactory}, this helper can use to 
validate index field
+     * type ans parse index format from pattern.
+     */
+    static class IndexHelper {
+        private static final Pattern dynamicIndexPattern = 
Pattern.compile("\\{[^\\{\\}]+\\}?");
+        private static final Pattern dynamicIndexTimeExtractPattern =
+                Pattern.compile(".*\\{.+\\|.*\\}.*");
+        private static final Pattern dynamicIndexSystemTimeExtractPattern =
+                Pattern.compile(
+                        
".*\\{\\s*(now\\(\\s*\\)|NOW\\(\\s*\\)|current_timestamp|CURRENT_TIMESTAMP)\\s*\\|.*\\}.*");
+        private static final List<LogicalTypeRoot> supportedTypes = new 
ArrayList<>();
+        private static final Map<LogicalTypeRoot, String> defaultFormats = new 
HashMap<>();
+
+        static {
+            // time related types
+            supportedTypes.add(LogicalTypeRoot.DATE);
+            supportedTypes.add(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE);
+            supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE);
+            supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE);

Review Comment:
   or comment this line out explicitly showing it's unsupported



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to