(flink) 02/02: [FLINK-36242][table] Drop all materialized tables manually instead of automatically dropping them in the after method.

2024-09-13 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 36103ff3a97fd7de656a795484999418c92f9eb5
Author: Feng Jin 
AuthorDate: Tue Sep 10 11:08:04 2024 +0800

[FLINK-36242][table] Drop all materialized tables manually instead of 
automatically dropping them in the after method.
---
 .../AbstractMaterializedTableStatementITCase.java  | 43 +---
 ...GatewayRestEndpointMaterializedTableITCase.java | 10 
 .../service/MaterializedTableStatementITCase.java  | 57 +-
 3 files changed, 77 insertions(+), 33 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractMaterializedTableStatementITCase.java
 
b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractMaterializedTableStatementITCase.java
index 1a6ee690cbd..7ec2aad15de 100644
--- 
a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractMaterializedTableStatementITCase.java
+++ 
b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractMaterializedTableStatementITCase.java
@@ -27,12 +27,10 @@ import org.apache.flink.runtime.client.JobStatusMessage;
 import org.apache.flink.runtime.jobgraph.JobType;
 import org.apache.flink.runtime.rest.messages.job.JobDetailsInfo;
 import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
-import org.apache.flink.table.catalog.CatalogBaseTable;
 import org.apache.flink.table.catalog.CatalogMaterializedTable;
-import org.apache.flink.table.catalog.ResolvedCatalogBaseTable;
+import org.apache.flink.table.catalog.ObjectIdentifier;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.gateway.api.operation.OperationHandle;
-import org.apache.flink.table.gateway.api.results.TableInfo;
 import org.apache.flink.table.gateway.api.session.SessionEnvironment;
 import org.apache.flink.table.gateway.api.session.SessionHandle;
 import org.apache.flink.table.gateway.api.utils.MockedEndpointVersion;
@@ -47,7 +45,6 @@ import 
org.apache.flink.testutils.executor.TestExecutorExtension;
 import org.apache.flink.types.Row;
 import org.apache.flink.util.concurrent.ExecutorThreadFactory;
 
-import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Order;
@@ -57,12 +54,10 @@ import org.junit.jupiter.api.io.TempDir;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.time.Duration;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
-import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
@@ -181,32 +176,6 @@ public abstract class 
AbstractMaterializedTableStatementITCase {
 restClusterClient = injectClusterClient;
 }
 
-@AfterEach
-void after() throws Exception {
-Set tableInfos =
-service.listTables(
-sessionHandle,
-fileSystemCatalogName,
-TEST_DEFAULT_DATABASE,
-
Collections.singleton(CatalogBaseTable.TableKind.TABLE));
-
-// drop all materialized tables
-for (TableInfo tableInfo : tableInfos) {
-ResolvedCatalogBaseTable resolvedTable =
-service.getTable(sessionHandle, tableInfo.getIdentifier());
-if (CatalogBaseTable.TableKind.MATERIALIZED_TABLE == 
resolvedTable.getTableKind()) {
-String dropTableDDL =
-String.format(
-"DROP MATERIALIZED TABLE %s",
-
tableInfo.getIdentifier().asSerializableString());
-OperationHandle dropTableHandle =
-service.executeStatement(
-sessionHandle, dropTableDDL, -1, new 
Configuration());
-awaitOperationTermination(service, sessionHandle, 
dropTableHandle);
-}
-}
-}
-
 private SessionHandle initializeSession() {
 SessionHandle sessionHandle = 
service.openSession(defaultSessionEnvironment);
 String catalogDDL =
@@ -364,4 +333,14 @@ public abstract class 
AbstractMaterializedTableStatementITCase {
 Duration.ofMillis(pause),
 "Failed to verify whether the job is finished.");
 }
+
+public void dropMaterializedTable(ObjectIdentifier objectIdentifier) 
throws Exception {
+String dropMaterializedTableDDL =
+String.format(
+"DROP MATERIALIZED TABLE %s", 
objectIdentifier.asSerializableString());
+OperationHandle dropMaterializedTableHandle =
+  

(flink) branch master updated (8d4e98503fd -> 36103ff3a97)

2024-09-13 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 8d4e98503fd [FLINK-36225][core] Remove deprecated method marked in 
FLIP-382
 new 5ea7ea70f31 [FLINK-36242][test] Fix inconsistent logic between 
listTable and getTable in TestFileSystemCatalog
 new 36103ff3a97 [FLINK-36242][table] Drop all materialized tables manually 
instead of automatically dropping them in the after method.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../AbstractMaterializedTableStatementITCase.java  | 43 +---
 ...GatewayRestEndpointMaterializedTableITCase.java | 10 
 .../service/MaterializedTableStatementITCase.java  | 57 +-
 .../testutils/catalog/TestFileSystemCatalog.java   |  9 ++--
 .../catalog/TestFileSystemCatalogTest.java | 15 +-
 .../catalog/TestFileSystemCatalogTestBase.java |  1 +
 6 files changed, 97 insertions(+), 38 deletions(-)



(flink) 01/02: [FLINK-36242][test] Fix inconsistent logic between listTable and getTable in TestFileSystemCatalog

2024-09-13 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 5ea7ea70f317bea1d544579ed6f9bac8b72d15ec
Author: Feng Jin 
AuthorDate: Tue Sep 10 11:06:23 2024 +0800

[FLINK-36242][test] Fix inconsistent logic between listTable and getTable 
in TestFileSystemCatalog
---
 .../file/testutils/catalog/TestFileSystemCatalog.java |  9 ++---
 .../file/testutils/catalog/TestFileSystemCatalogTest.java | 15 +--
 .../testutils/catalog/TestFileSystemCatalogTestBase.java  |  1 +
 3 files changed, 20 insertions(+), 5 deletions(-)

diff --git 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalog.java
 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalog.java
index 6d64ecee032..3d72da3ae78 100644
--- 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalog.java
+++ 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalog.java
@@ -241,6 +241,7 @@ public class TestFileSystemCatalog extends AbstractCatalog {
 return Arrays.stream(fs.listStatus(dbPath))
 .filter(FileStatus::isDir)
 .map(fileStatus -> fileStatus.getPath().getName())
+.filter(name -> tableExists(new ObjectPath(databaseName, 
name)))
 .collect(Collectors.toList());
 } catch (IOException e) {
 throw new CatalogException(
@@ -355,10 +356,12 @@ public class TestFileSystemCatalog extends 
AbstractCatalog {
 try {
 if (!fs.exists(path)) {
 fs.mkdirs(path);
+}
+if (!fs.exists(tableSchemaPath)) {
 fs.mkdirs(tableSchemaPath);
-if (isFileSystemTable(catalogTable.getOptions())) {
-fs.mkdirs(tableDataPath);
-}
+}
+if (isFileSystemTable(catalogTable.getOptions()) && 
!fs.exists(tableDataPath)) {
+fs.mkdirs(tableDataPath);
 }
 
 // write table schema
diff --git 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/test/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalogTest.java
 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/test/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalogTest.java
index 3d7ba88a96e..1936896146b 100644
--- 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/test/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalogTest.java
+++ 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/test/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalogTest.java
@@ -42,6 +42,8 @@ import org.apache.flink.table.refresh.RefreshHandler;
 
 import org.junit.jupiter.api.Test;
 
+import java.nio.file.Files;
+import java.nio.file.Paths;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
@@ -305,12 +307,21 @@ public class TestFileSystemCatalogTest extends 
TestFileSystemCatalogTestBase {
 
 // test list table
 List tables = catalog.listTables(TEST_DEFAULT_DATABASE);
-assertThat(tables.contains(tablePath1.getObjectName())).isTrue();
-assertThat(tables.contains(tablePath2.getObjectName())).isTrue();
+assertThat(tables).contains(tablePath1.getObjectName());
+assertThat(tables).contains(tablePath2.getObjectName());
 
 // test list non-exist database table
 assertThrows(
 DatabaseNotExistException.class, () -> 
catalog.listTables(NONE_EXIST_DATABASE));
+
+// test list table ignore the empty path
+String nonExistTablePath =
+String.format(
+"%s/%s/%s",
+tempFile.getAbsolutePath(), TEST_DEFAULT_DATABASE, 
NONE_EXIST_TABLE);
+Files.createDirectories(Paths.get(nonExistTablePath));
+tables = catalog.listTables(TEST_DEFAULT_DATABASE);
+assertThat(tables).doesNotContain(NONE_EXIST_TABLE);
 }
 
 @Test
diff --git 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/test/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalogTestBase.java
 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/test/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalogTestBase.java
index 8cd5fc306e6..fb419cf019e 100644
--- 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/test/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalogTestBase.java
+++ 
b/flink-te

(flink) branch master updated: [FLINK-35910][table] Add the built-in function BTRIM

2024-08-07 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 72ebee2eb45 [FLINK-35910][table] Add the built-in function BTRIM
72ebee2eb45 is described below

commit 72ebee2eb45f598ebd88422f716ae6da93ddee5d
Author: dylanhz <53137516+dyla...@users.noreply.github.com>
AuthorDate: Thu Aug 8 10:32:24 2024 +0800

[FLINK-35910][table] Add the built-in function BTRIM

This closes #25127

-

Co-authored-by: Ron 
---
 docs/data/sql_functions.yml|  6 ++
 docs/data/sql_functions_zh.yml |  6 ++
 .../docs/reference/pyflink.table/expressions.rst   |  1 +
 flink-python/pyflink/table/expression.py   | 10 +++
 .../flink/table/api/internal/BaseExpressions.java  | 12 
 .../functions/BuiltInFunctionDefinitions.java  | 19 ++
 .../planner/functions/StringFunctionsITCase.java   | 74 +-
 .../runtime/functions/scalar/BTrimFunction.java| 51 +++
 8 files changed, 178 insertions(+), 1 deletion(-)

diff --git a/docs/data/sql_functions.yml b/docs/data/sql_functions.yml
index 2cd2a3c415f..8dca117fa7b 100644
--- a/docs/data/sql_functions.yml
+++ b/docs/data/sql_functions.yml
@@ -274,6 +274,12 @@ string:
   - sql: RTRIM(string)
 table: STRING.rtrim()
 description: Returns a string that removes the right whitespaces from 
STRING. E.g., 'This is a test String. '.rtrim() returns "This is a test 
String.".
+  - sql: BTRIM(str[, trimStr])
+table: str.btrim([trimStr])
+description: |
+  Removes any leading and trailing characters within trimStr from str. 
trimStr is set to whitespace by default.
+  str , trimStr 
+  Returns a STRING representation of the trimmed str. `NULL` if any of the 
arguments are `NULL`.
   - sql: REPEAT(string, int)
 table: STRING.repeat(INT)
 description: Returns a string that repeats the base string integer times. 
E.g., REPEAT('This is a test String.', 2) returns "This is a test String.This 
is a test String.".
diff --git a/docs/data/sql_functions_zh.yml b/docs/data/sql_functions_zh.yml
index 192a06c302a..ad8451b13f2 100644
--- a/docs/data/sql_functions_zh.yml
+++ b/docs/data/sql_functions_zh.yml
@@ -340,6 +340,12 @@ string:
 description: |
   返回从 STRING 中删除右边空格的字符串。
   例如 `'This is a test String. '.rtrim()` 返回 `'This is a test String.'`。
+  - sql: BTRIM(str[, trimStr])
+table: str.btrim([trimStr])
+description: |
+  从 str 的左边和右边删除 trimStr 中的字符。trimStr 默认设置为空格。  
+  str , trimStr 
+  返回一个 STRING 格式的裁剪后的 str。如果任何参数为 `NULL`,则返回 `NULL`。
   - sql: REPEAT(string, int)
 table: STRING.repeat(INT)
 description: |
diff --git a/flink-python/docs/reference/pyflink.table/expressions.rst 
b/flink-python/docs/reference/pyflink.table/expressions.rst
index 3a41b16df26..57c8ed314c8 100644
--- a/flink-python/docs/reference/pyflink.table/expressions.rst
+++ b/flink-python/docs/reference/pyflink.table/expressions.rst
@@ -194,6 +194,7 @@ string functions
 Expression.parse_url
 Expression.ltrim
 Expression.rtrim
+Expression.btrim
 Expression.repeat
 Expression.over
 Expression.reverse
diff --git a/flink-python/pyflink/table/expression.py 
b/flink-python/pyflink/table/expression.py
index ebaed656d23..a8500836514 100644
--- a/flink-python/pyflink/table/expression.py
+++ b/flink-python/pyflink/table/expression.py
@@ -1322,6 +1322,16 @@ class Expression(Generic[T]):
 """
 return _unary_op("rtrim")(self)
 
+def btrim(self, trim_str=None) -> 'Expression':
+"""
+Removes any leading and trailing characters within trim_str from str.
+trim_str is set to whitespace by default.
+"""
+if trim_str is None:
+return _unary_op("btrim")(self)
+else:
+return _binary_op("btrim")(self, trim_str)
+
 def repeat(self, n: Union[int, 'Expression[int]']) -> 'Expression[str]':
 """
 Returns a string that repeats the base string n times.
diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
index 6dfcaf93f0d..1a201d99547 100644
--- 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
+++ 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
@@ -76,6 +76,7 @@ import static 
org.apache.flink.table.functions.BuiltInFunctionDefin

(flink) branch release-1.20 updated: [FLINK-35872][table] Fix the incorrect partition generation for materialized table period refresh in Full Mode

2024-07-22 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.20 by this push:
 new a3268544636 [FLINK-35872][table] Fix the incorrect partition 
generation for materialized table period refresh in Full Mode
a3268544636 is described below

commit a3268544636ee120dd6cbed1ba429aafdca2909a
Author: Feng Jin 
AuthorDate: Sun Jul 21 22:02:34 2024 +0800

[FLINK-35872][table] Fix the incorrect partition generation for 
materialized table period refresh in Full Mode

This closes #25108

(cherry picked from commit af7d2b3ab0e0ecc4157005704bebac9c767f2e1a)
---
 .../MaterializedTableManager.java  |  14 +-
 .../service/MaterializedTableStatementITCase.java  |  11 +-
 .../MaterializedTableManagerTest.java  | 405 ++---
 .../apache/flink/table/utils/DateTimeUtils.java|  15 +-
 4 files changed, 388 insertions(+), 57 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
index aa0877265a0..2398e72172a 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
@@ -28,6 +28,7 @@ import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.api.config.TableConfigOptions;
 import org.apache.flink.table.catalog.CatalogMaterializedTable;
 import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.IntervalFreshness;
 import org.apache.flink.table.catalog.ObjectIdentifier;
 import org.apache.flink.table.catalog.ResolvedCatalogBaseTable;
 import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable;
@@ -102,8 +103,9 @@ import static 
org.apache.flink.table.factories.WorkflowSchedulerFactoryUtil.WORK
 import static 
org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpointFactoryUtils.getEndpointConfig;
 import static 
org.apache.flink.table.gateway.service.utils.Constants.CLUSTER_INFO;
 import static org.apache.flink.table.gateway.service.utils.Constants.JOB_ID;
-import static org.apache.flink.table.utils.DateTimeUtils.formatTimestampString;
+import static 
org.apache.flink.table.utils.DateTimeUtils.formatTimestampStringWithOffset;
 import static 
org.apache.flink.table.utils.IntervalFreshnessUtils.convertFreshnessToCron;
+import static 
org.apache.flink.table.utils.IntervalFreshnessUtils.convertFreshnessToDuration;
 
 /** Manager is responsible for execute the {@link MaterializedTableOperation}. 
*/
 @Internal
@@ -615,6 +617,7 @@ public class MaterializedTableManager {
 isPeriodic
 ? getPeriodRefreshPartition(
 scheduleTime,
+materializedTable.getDefinitionFreshness(),
 materializedTableIdentifier,
 materializedTable.getOptions(),
 operationExecutor
@@ -686,13 +689,14 @@ public class MaterializedTableManager {
 @VisibleForTesting
 static Map getPeriodRefreshPartition(
 String scheduleTime,
+IntervalFreshness freshness,
 ObjectIdentifier materializedTableIdentifier,
 Map materializedTableOptions,
 ZoneId localZoneId) {
 if (scheduleTime == null) {
 throw new ValidationException(
 String.format(
-"Scheduler time not properly set for periodic 
refresh of materialized table %s.",
+"The scheduler time must not be null during the 
periodic refresh of the materialized table %s.",
 materializedTableIdentifier));
 }
 
@@ -707,12 +711,14 @@ public class MaterializedTableManager {
 PARTITION_FIELDS.length() + 1,
 partKey.length() - (DATE_FORMATTER.length() + 1));
 String partFieldFormatter = materializedTableOptions.get(partKey);
+
 String partFiledValue =
-formatTimestampString(
+formatTimestampStringWithOffset(
 scheduleTime,
 SCHEDULE_TIME_DATE_FORMATTER_DEFAULT,
 partFieldFormatter,
-TimeZone.getTimeZone(localZoneId));
+TimeZone.getTimeZone(localZoneId),
+-convertFreshnessToDuration

(flink) branch master updated: [FLINK-35872][table] Fix the incorrect partition generation for materialized table period refresh in Full Mode

2024-07-22 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new af7d2b3ab0e [FLINK-35872][table] Fix the incorrect partition 
generation for materialized table period refresh in Full Mode
af7d2b3ab0e is described below

commit af7d2b3ab0e0ecc4157005704bebac9c767f2e1a
Author: Feng Jin 
AuthorDate: Sun Jul 21 22:02:34 2024 +0800

[FLINK-35872][table] Fix the incorrect partition generation for 
materialized table period refresh in Full Mode

This closes #25108
---
 .../MaterializedTableManager.java  |  14 +-
 .../service/MaterializedTableStatementITCase.java  |  11 +-
 .../MaterializedTableManagerTest.java  | 405 ++---
 .../apache/flink/table/utils/DateTimeUtils.java|  15 +-
 4 files changed, 388 insertions(+), 57 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
index aa0877265a0..2398e72172a 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
@@ -28,6 +28,7 @@ import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.api.config.TableConfigOptions;
 import org.apache.flink.table.catalog.CatalogMaterializedTable;
 import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.IntervalFreshness;
 import org.apache.flink.table.catalog.ObjectIdentifier;
 import org.apache.flink.table.catalog.ResolvedCatalogBaseTable;
 import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable;
@@ -102,8 +103,9 @@ import static 
org.apache.flink.table.factories.WorkflowSchedulerFactoryUtil.WORK
 import static 
org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpointFactoryUtils.getEndpointConfig;
 import static 
org.apache.flink.table.gateway.service.utils.Constants.CLUSTER_INFO;
 import static org.apache.flink.table.gateway.service.utils.Constants.JOB_ID;
-import static org.apache.flink.table.utils.DateTimeUtils.formatTimestampString;
+import static 
org.apache.flink.table.utils.DateTimeUtils.formatTimestampStringWithOffset;
 import static 
org.apache.flink.table.utils.IntervalFreshnessUtils.convertFreshnessToCron;
+import static 
org.apache.flink.table.utils.IntervalFreshnessUtils.convertFreshnessToDuration;
 
 /** Manager is responsible for execute the {@link MaterializedTableOperation}. 
*/
 @Internal
@@ -615,6 +617,7 @@ public class MaterializedTableManager {
 isPeriodic
 ? getPeriodRefreshPartition(
 scheduleTime,
+materializedTable.getDefinitionFreshness(),
 materializedTableIdentifier,
 materializedTable.getOptions(),
 operationExecutor
@@ -686,13 +689,14 @@ public class MaterializedTableManager {
 @VisibleForTesting
 static Map getPeriodRefreshPartition(
 String scheduleTime,
+IntervalFreshness freshness,
 ObjectIdentifier materializedTableIdentifier,
 Map materializedTableOptions,
 ZoneId localZoneId) {
 if (scheduleTime == null) {
 throw new ValidationException(
 String.format(
-"Scheduler time not properly set for periodic 
refresh of materialized table %s.",
+"The scheduler time must not be null during the 
periodic refresh of the materialized table %s.",
 materializedTableIdentifier));
 }
 
@@ -707,12 +711,14 @@ public class MaterializedTableManager {
 PARTITION_FIELDS.length() + 1,
 partKey.length() - (DATE_FORMATTER.length() + 1));
 String partFieldFormatter = materializedTableOptions.get(partKey);
+
 String partFiledValue =
-formatTimestampString(
+formatTimestampStringWithOffset(
 scheduleTime,
 SCHEDULE_TIME_DATE_FORMATTER_DEFAULT,
 partFieldFormatter,
-TimeZone.getTimeZone(localZoneId));
+TimeZone.getTimeZone(localZoneId),
+-convertFreshnessToDuration(freshness).toMillis());
 if (partFiledValue == null) {
  

(flink) branch release-1.20 updated: [FLINK-35832][Table SQL / Planner] Fix IFNULL function returns incorrect result

2024-07-19 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.20 by this push:
 new eaefacef0d2 [FLINK-35832][Table SQL / Planner] Fix IFNULL function 
returns incorrect result
eaefacef0d2 is described below

commit eaefacef0d25d570b8ce8bcbbbc0a425196f6f79
Author: dylanhz <53137516+dyla...@users.noreply.github.com>
AuthorDate: Thu Jul 18 19:35:42 2024 +0800

[FLINK-35832][Table SQL / Planner] Fix IFNULL function returns incorrect 
result

This closes #25099
---
 .../types/inference/strategies/IfNullTypeStrategy.java | 18 --
 .../table/planner/functions/MiscFunctionsITCase.java   | 12 ++--
 .../planner/expressions/ScalarFunctionsTest.scala  | 16 
 .../table/planner/runtime/batch/sql/CalcITCase.scala   |  6 ++
 .../table/planner/runtime/stream/sql/CalcITCase.scala  | 14 +-
 5 files changed, 53 insertions(+), 13 deletions(-)

diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/IfNullTypeStrategy.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/IfNullTypeStrategy.java
index f3f09564439..f405cdc4002 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/IfNullTypeStrategy.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/IfNullTypeStrategy.java
@@ -22,11 +22,19 @@ import org.apache.flink.annotation.Internal;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.inference.CallContext;
 import org.apache.flink.table.types.inference.TypeStrategy;
+import org.apache.flink.table.types.logical.utils.LogicalTypeMerging;
+import org.apache.flink.table.types.utils.TypeConversions;
 
+import java.util.Arrays;
 import java.util.List;
 import java.util.Optional;
 
-/** Type strategy specific for avoiding nulls. */
+/**
+ * Type strategy specific for avoiding nulls. 
+ * If arg0 is non-nullable, output datatype is exactly the datatype of arg0. 
Otherwise, output
+ * datatype is the common type of arg0 and arg1. In the second case, output 
type is nullable only if
+ * both args are nullable.
+ */
 @Internal
 class IfNullTypeStrategy implements TypeStrategy {
 
@@ -35,10 +43,16 @@ class IfNullTypeStrategy implements TypeStrategy {
 final List argumentDataTypes = 
callContext.getArgumentDataTypes();
 final DataType inputDataType = argumentDataTypes.get(0);
 final DataType nullReplacementDataType = argumentDataTypes.get(1);
+
 if (!inputDataType.getLogicalType().isNullable()) {
 return Optional.of(inputDataType);
 }
 
-return Optional.of(nullReplacementDataType);
+return LogicalTypeMerging.findCommonType(
+Arrays.asList(
+inputDataType.getLogicalType(),
+nullReplacementDataType.getLogicalType()))
+.map(t -> 
t.copy(nullReplacementDataType.getLogicalType().isNullable()))
+.map(TypeConversions::fromLogicalToDataType);
 }
 }
diff --git 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
index 79020e42539..a9d243656b5 100644
--- 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
+++ 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
@@ -55,8 +55,11 @@ class MiscFunctionsITCase extends BuiltInFunctionTestBase {
 DataTypes.STRING())
 .testSqlResult("TYPEOF(NULL)", "NULL", 
DataTypes.STRING()),
 TestSetSpec.forFunction(BuiltInFunctionDefinitions.IF_NULL)
-.onFieldsWithData(null, new BigDecimal("123.45"))
-.andDataTypes(DataTypes.INT().nullable(), 
DataTypes.DECIMAL(5, 2).notNull())
+.onFieldsWithData(null, new BigDecimal("123.45"), 
"Hello world")
+.andDataTypes(
+DataTypes.INT().nullable(),
+DataTypes.DECIMAL(5, 2).notNull(),
+DataTypes.STRING())
 .withFunction(TakesNotNull.class)
 .testResult(
 $("f0").ifNull($("f0")),
@@ -81,6 +84,11 @@ class MiscFunctionsITCase extends BuiltInFunctionTestBase {
   

(flink) branch release-1.19 updated: [FLINK-35832][Table SQL / Planner] Fix IFNULL function returns incorrect result (#25106)

2024-07-19 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.19
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.19 by this push:
 new 78ba7c9adb3 [FLINK-35832][Table SQL / Planner] Fix IFNULL function 
returns incorrect result (#25106)
78ba7c9adb3 is described below

commit 78ba7c9adb3571ee934fad8cb1888025f32b24d0
Author: dylanhz <53137516+dyla...@users.noreply.github.com>
AuthorDate: Fri Jul 19 20:16:04 2024 +0800

[FLINK-35832][Table SQL / Planner] Fix IFNULL function returns incorrect 
result (#25106)

This closes #25099

(cherry picked from commit 6719efc6b12af62bab93b1a7997a1fe5eab97017)
---
 .../types/inference/strategies/IfNullTypeStrategy.java | 18 --
 .../table/planner/functions/MiscFunctionsITCase.java   | 12 ++--
 .../planner/expressions/ScalarFunctionsTest.scala  | 16 
 .../table/planner/runtime/batch/sql/CalcITCase.scala   |  6 ++
 .../table/planner/runtime/stream/sql/CalcITCase.scala  | 14 +-
 5 files changed, 53 insertions(+), 13 deletions(-)

diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/IfNullTypeStrategy.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/IfNullTypeStrategy.java
index f3f09564439..f405cdc4002 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/IfNullTypeStrategy.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/IfNullTypeStrategy.java
@@ -22,11 +22,19 @@ import org.apache.flink.annotation.Internal;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.inference.CallContext;
 import org.apache.flink.table.types.inference.TypeStrategy;
+import org.apache.flink.table.types.logical.utils.LogicalTypeMerging;
+import org.apache.flink.table.types.utils.TypeConversions;
 
+import java.util.Arrays;
 import java.util.List;
 import java.util.Optional;
 
-/** Type strategy specific for avoiding nulls. */
+/**
+ * Type strategy specific for avoiding nulls. 
+ * If arg0 is non-nullable, output datatype is exactly the datatype of arg0. 
Otherwise, output
+ * datatype is the common type of arg0 and arg1. In the second case, output 
type is nullable only if
+ * both args are nullable.
+ */
 @Internal
 class IfNullTypeStrategy implements TypeStrategy {
 
@@ -35,10 +43,16 @@ class IfNullTypeStrategy implements TypeStrategy {
 final List argumentDataTypes = 
callContext.getArgumentDataTypes();
 final DataType inputDataType = argumentDataTypes.get(0);
 final DataType nullReplacementDataType = argumentDataTypes.get(1);
+
 if (!inputDataType.getLogicalType().isNullable()) {
 return Optional.of(inputDataType);
 }
 
-return Optional.of(nullReplacementDataType);
+return LogicalTypeMerging.findCommonType(
+Arrays.asList(
+inputDataType.getLogicalType(),
+nullReplacementDataType.getLogicalType()))
+.map(t -> 
t.copy(nullReplacementDataType.getLogicalType().isNullable()))
+.map(TypeConversions::fromLogicalToDataType);
 }
 }
diff --git 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
index 79020e42539..a9d243656b5 100644
--- 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
+++ 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
@@ -55,8 +55,11 @@ class MiscFunctionsITCase extends BuiltInFunctionTestBase {
 DataTypes.STRING())
 .testSqlResult("TYPEOF(NULL)", "NULL", 
DataTypes.STRING()),
 TestSetSpec.forFunction(BuiltInFunctionDefinitions.IF_NULL)
-.onFieldsWithData(null, new BigDecimal("123.45"))
-.andDataTypes(DataTypes.INT().nullable(), 
DataTypes.DECIMAL(5, 2).notNull())
+.onFieldsWithData(null, new BigDecimal("123.45"), 
"Hello world")
+.andDataTypes(
+DataTypes.INT().nullable(),
+DataTypes.DECIMAL(5, 2).notNull(),
+DataTypes.STRING())
 .withFunction(TakesNotNull.class)
 .testResult(
 $("f0").ifNull($("f0")

(flink) branch master updated: [FLINK-35832][Table SQL / Planner] Fix IFNULL function returns incorrect result

2024-07-18 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 6719efc6b12 [FLINK-35832][Table SQL / Planner] Fix IFNULL function 
returns incorrect result
6719efc6b12 is described below

commit 6719efc6b12af62bab93b1a7997a1fe5eab97017
Author: dylanhz <53137516+dyla...@users.noreply.github.com>
AuthorDate: Thu Jul 18 19:35:42 2024 +0800

[FLINK-35832][Table SQL / Planner] Fix IFNULL function returns incorrect 
result

This closes #25099
---
 .../types/inference/strategies/IfNullTypeStrategy.java | 18 --
 .../table/planner/functions/MiscFunctionsITCase.java   | 12 ++--
 .../planner/expressions/ScalarFunctionsTest.scala  | 16 
 .../table/planner/runtime/batch/sql/CalcITCase.scala   |  6 ++
 .../table/planner/runtime/stream/sql/CalcITCase.scala  | 14 +-
 5 files changed, 53 insertions(+), 13 deletions(-)

diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/IfNullTypeStrategy.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/IfNullTypeStrategy.java
index f3f09564439..f405cdc4002 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/IfNullTypeStrategy.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/IfNullTypeStrategy.java
@@ -22,11 +22,19 @@ import org.apache.flink.annotation.Internal;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.inference.CallContext;
 import org.apache.flink.table.types.inference.TypeStrategy;
+import org.apache.flink.table.types.logical.utils.LogicalTypeMerging;
+import org.apache.flink.table.types.utils.TypeConversions;
 
+import java.util.Arrays;
 import java.util.List;
 import java.util.Optional;
 
-/** Type strategy specific for avoiding nulls. */
+/**
+ * Type strategy specific for avoiding nulls. 
+ * If arg0 is non-nullable, output datatype is exactly the datatype of arg0. 
Otherwise, output
+ * datatype is the common type of arg0 and arg1. In the second case, output 
type is nullable only if
+ * both args are nullable.
+ */
 @Internal
 class IfNullTypeStrategy implements TypeStrategy {
 
@@ -35,10 +43,16 @@ class IfNullTypeStrategy implements TypeStrategy {
 final List argumentDataTypes = 
callContext.getArgumentDataTypes();
 final DataType inputDataType = argumentDataTypes.get(0);
 final DataType nullReplacementDataType = argumentDataTypes.get(1);
+
 if (!inputDataType.getLogicalType().isNullable()) {
 return Optional.of(inputDataType);
 }
 
-return Optional.of(nullReplacementDataType);
+return LogicalTypeMerging.findCommonType(
+Arrays.asList(
+inputDataType.getLogicalType(),
+nullReplacementDataType.getLogicalType()))
+.map(t -> 
t.copy(nullReplacementDataType.getLogicalType().isNullable()))
+.map(TypeConversions::fromLogicalToDataType);
 }
 }
diff --git 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
index 79020e42539..a9d243656b5 100644
--- 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
+++ 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
@@ -55,8 +55,11 @@ class MiscFunctionsITCase extends BuiltInFunctionTestBase {
 DataTypes.STRING())
 .testSqlResult("TYPEOF(NULL)", "NULL", 
DataTypes.STRING()),
 TestSetSpec.forFunction(BuiltInFunctionDefinitions.IF_NULL)
-.onFieldsWithData(null, new BigDecimal("123.45"))
-.andDataTypes(DataTypes.INT().nullable(), 
DataTypes.DECIMAL(5, 2).notNull())
+.onFieldsWithData(null, new BigDecimal("123.45"), 
"Hello world")
+.andDataTypes(
+DataTypes.INT().nullable(),
+DataTypes.DECIMAL(5, 2).notNull(),
+DataTypes.STRING())
 .withFunction(TakesNotNull.class)
 .testResult(
 $("f0").ifNull($("f0")),
@@ -81,6 +84,11 @@ class MiscFunctionsITCase extends BuiltInFunctionTestBase {
   

(flink) 02/02: [FLINK-35754][doc] Fix typo in materialized table docs

2024-07-11 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 8244941ed18c9c063449759927b58b629110d89e
Author: Feng Jin 
AuthorDate: Mon Jul 8 10:06:36 2024 +0800

[FLINK-35754][doc] Fix typo in materialized table docs

(cherry picked from commit d04c70fae033bb04ed7a3bb69832be55b4425700)
---
 docs/content.zh/docs/dev/table/materialized-table/statements.md | 2 +-
 docs/content/docs/dev/table/materialized-table/overview.md  | 4 ++--
 docs/content/docs/dev/table/materialized-table/statements.md| 8 
 3 files changed, 7 insertions(+), 7 deletions(-)

diff --git a/docs/content.zh/docs/dev/table/materialized-table/statements.md 
b/docs/content.zh/docs/dev/table/materialized-table/statements.md
index ceda0c92364..1838845d708 100644
--- a/docs/content.zh/docs/dev/table/materialized-table/statements.md
+++ b/docs/content.zh/docs/dev/table/materialized-table/statements.md
@@ -96,7 +96,7 @@ CREATE MATERIALIZED TABLE my_materialized_table
 如上例所示,我们为 `ds` 分区列指定了 `date-formatter` 选项。每次调度时,调度时间将转换为相应的 `ds` 分区值。例如,在 
`2024-01-01 00:00:00` 的调度时间下,只有分区 `ds = '2024-01-01'` 会被刷新。
 
 注意
-- `partition.fields.#.date-formatter` 选项仅适用于全量模式。
+- [partition.fields.#.date-formatter]({{< ref "docs/dev/table/config" 
>}}#partition-fields-date-formatter) 选项仅适用于全量模式。
 - [partition.fields.#.date-formatter]({{< ref "docs/dev/table/config" 
>}}#partition-fields-date-formatter) 中的字段必须是有效的字符串类型分区字段。
 
 ## FRESHNESS
diff --git a/docs/content/docs/dev/table/materialized-table/overview.md 
b/docs/content/docs/dev/table/materialized-table/overview.md
index a2e6c830773..d49492e314a 100644
--- a/docs/content/docs/dev/table/materialized-table/overview.md
+++ b/docs/content/docs/dev/table/materialized-table/overview.md
@@ -45,8 +45,8 @@ Data freshness is a crucial attribute of a materialized 
table, serving two main
 - CONTINUOUS mode: Launches a Flink streaming job that continuously 
refreshes the materialized table data.
 - FULL mode: The workflow scheduler periodically triggers a Flink batch 
job to refresh the materialized table data.
 - **Determining the Refresh Frequency**.
-- In CONTINUOUS mode, data freshness is converted into the `checkpoint` 
interval of the Flink streaming job currently.
-- In FULL mode, data freshness is converted into the scheduling cycle of 
the workflow, e.g. cron expression.
+- In CONTINUOUS mode, data freshness is converted into the `checkpoint` 
interval of the Flink streaming job.
+- In FULL mode, data freshness is converted into the scheduling cycle of 
the workflow, e.g., a cron expression.
 
 ## Refresh Mode
 
diff --git a/docs/content/docs/dev/table/materialized-table/statements.md 
b/docs/content/docs/dev/table/materialized-table/statements.md
index 18d90675bb9..490652d0732 100644
--- a/docs/content/docs/dev/table/materialized-table/statements.md
+++ b/docs/content/docs/dev/table/materialized-table/statements.md
@@ -28,7 +28,7 @@ under the License.
 
 Flink SQL supports the following Materialized Table statements for now:
 - [CREATE MATERIALIZED TABLE](#create-materialized-table)
-- [Alter MATERIALIZED TABLE](#alter-materialized-table)
+- [ALTER MATERIALIZED TABLE](#alter-materialized-table)
 - [DROP MATERIALIZED TABLE](#drop-materialized-table)
 
 # CREATE MATERIALIZED TABLE
@@ -60,7 +60,7 @@ AS 
 
 ## PARTITIONED BY
 
-`PARTITIONED BY` define an optional list of columns to partition the 
materialized table. A directory is created for each partition if this 
materialized table is used as a filesystem sink.
+`PARTITIONED BY` defines an optional list of columns to partition the 
materialized table. A directory is created for each partition if this 
materialized table is used as a filesystem sink.
 
 **Example:**
 
@@ -96,12 +96,12 @@ CREATE MATERIALIZED TABLE my_materialized_table
 As shown in the above example, we specified the date-formatter option for the 
`ds` partition column. During each scheduling, the scheduling time will be 
converted to the ds partition value. For example, for a scheduling time of 
`2024-01-01 00:00:00`, only the partition `ds = '2024-01-01'` will be refreshed.
 
 Note
-- The `partition.fields.#.date-formatter` option only works in full mode.
+- The [partition.fields.#.date-formatter]({{< ref "docs/dev/table/config" 
>}}#partition-fields-date-formatter) option only works in full mode.
 - The field in the [partition.fields.#.date-formatter]({{< ref 
"docs/dev/table/config" >}}#partition-fields-date-formatter) must be a valid 
string type partition field.
 
 ## FRESHNESS
 
-`FRESHNESS` define the data freshness of a materialized table.
+`FRESHNESS` defines the data freshness of a materialized table.
 
 **FRESHNESS and Refresh Mode Relationship**
 



(flink) branch release-1.20 updated (c8b46b09087 -> 8244941ed18)

2024-07-11 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git


from c8b46b09087 [FLINK-35801] Fix SnapshotFileMergingCompatibilityITCase, 
wait for file deletion before quit (#25067)
 new 7d9d68e634b [FLINK-35754][e2e] Fix 
SqlGatewayE2ECase.testMaterializedTableInFullMode failed due to Internal Server 
Error
 new 8244941ed18 [FLINK-35754][doc] Fix typo in materialized table docs

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../dev/table/materialized-table/statements.md |  2 +-
 .../docs/dev/table/materialized-table/overview.md  |  4 +--
 .../dev/table/materialized-table/statements.md |  8 +++---
 .../flink/table/gateway/SqlGatewayE2ECase.java | 13 +
 .../flink/core/testutils/CommonTestUtils.java  | 33 ++
 5 files changed, 47 insertions(+), 13 deletions(-)



(flink) 01/02: [FLINK-35754][e2e] Fix SqlGatewayE2ECase.testMaterializedTableInFullMode failed due to Internal Server Error

2024-07-11 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 7d9d68e634bfa3dc0cc8ac127cc130ee0a83df54
Author: Feng Jin 
AuthorDate: Fri Jul 5 15:56:52 2024 +0800

[FLINK-35754][e2e] Fix SqlGatewayE2ECase.testMaterializedTableInFullMode 
failed due to Internal Server Error

(cherry picked from commit 140d96c00410d16d56f293b321e73475532878c6)
---
 .../flink/table/gateway/SqlGatewayE2ECase.java | 13 +
 .../flink/core/testutils/CommonTestUtils.java  | 33 ++
 2 files changed, 40 insertions(+), 6 deletions(-)

diff --git 
a/flink-end-to-end-tests/flink-sql-gateway-test/src/test/java/org/apache/flink/table/gateway/SqlGatewayE2ECase.java
 
b/flink-end-to-end-tests/flink-sql-gateway-test/src/test/java/org/apache/flink/table/gateway/SqlGatewayE2ECase.java
index d65ed2f5f5d..0d7387b5803 100644
--- 
a/flink-end-to-end-tests/flink-sql-gateway-test/src/test/java/org/apache/flink/table/gateway/SqlGatewayE2ECase.java
+++ 
b/flink-end-to-end-tests/flink-sql-gateway-test/src/test/java/org/apache/flink/table/gateway/SqlGatewayE2ECase.java
@@ -187,7 +187,7 @@ public class SqlGatewayE2ECase extends TestLogger {
 gatewayRestClient.executeStatementWithResult("SET 
'execution.runtime-mode' = 'batch'");
 
 // verify the result
-CommonTestUtils.waitUtil(
+CommonTestUtils.waitUntilIgnoringExceptions(
 () -> {
 List result =
 gatewayRestClient.executeStatementWithResult(
@@ -219,7 +219,7 @@ public class SqlGatewayE2ECase extends TestLogger {
 "ALTER MATERIALIZED TABLE 
my_materialized_table_in_continuous_mode RESUME");
 
 // verify the result
-CommonTestUtils.waitUtil(
+CommonTestUtils.waitUntilIgnoringExceptions(
 () -> {
 List result =
 gatewayRestClient.executeStatementWithResult(
@@ -312,7 +312,7 @@ public class SqlGatewayE2ECase extends TestLogger {
 
 // verify the materialized table should auto refresh the today 
partition or tomorrow
 // partition
-CommonTestUtils.waitUtil(
+CommonTestUtils.waitUntilIgnoringExceptions(
 () -> {
 List result =
 gatewayRestClient.executeStatementWithResult(
@@ -345,9 +345,10 @@ public class SqlGatewayE2ECase extends TestLogger {
 gatewayRestClient.executeStatementWithResult(
 "ALTER MATERIALIZED TABLE 
my_materialized_table_in_full_mode RESUME");
 
-// wait until the materialized table is updated and verify only 
today or tomorrow data
+// wait until the materialized table is updated and verify only 
today or tomorrow
+// data
 // should be updated
-CommonTestUtils.waitUtil(
+CommonTestUtils.waitUntilIgnoringExceptions(
 () -> {
 List result =
 gatewayRestClient.executeStatementWithResult(
@@ -378,7 +379,7 @@ public class SqlGatewayE2ECase extends TestLogger {
 + "')");
 
 // verify the materialized table that all partitions are updated
-CommonTestUtils.waitUtil(
+CommonTestUtils.waitUntilIgnoringExceptions(
 () -> {
 List result =
 gatewayRestClient.executeStatementWithResult(
diff --git 
a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java
 
b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java
index 7466648a904..70e474e9e86 100644
--- 
a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java
+++ 
b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java
@@ -19,6 +19,8 @@
 package org.apache.flink.core.testutils;
 
 import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.BufferedWriter;
 import java.io.ByteArrayInputStream;
@@ -42,6 +44,8 @@ import static org.hamcrest.MatcherAssert.assertThat;
 /** This class contains reusable utility methods for unit tests. */
 public class CommonTestUtils {
 
+private static final Logger LOG = 
LoggerFactory.getLogger(CommonTestUtils.class);
+
 /**
  * Creates a copy of an object via Java Serialization.
  *
@@ -215,6 +219,35 @@ public class CommonTestUtils {
 }
 }
 
+/**
+ * Wait until the given condi

(flink) 02/02: [FLINK-35754][doc] Fix typo in materialized table docs

2024-07-10 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit d04c70fae033bb04ed7a3bb69832be55b4425700
Author: Feng Jin 
AuthorDate: Mon Jul 8 10:06:36 2024 +0800

[FLINK-35754][doc] Fix typo in materialized table docs
---
 docs/content.zh/docs/dev/table/materialized-table/statements.md | 2 +-
 docs/content/docs/dev/table/materialized-table/overview.md  | 4 ++--
 docs/content/docs/dev/table/materialized-table/statements.md| 8 
 3 files changed, 7 insertions(+), 7 deletions(-)

diff --git a/docs/content.zh/docs/dev/table/materialized-table/statements.md 
b/docs/content.zh/docs/dev/table/materialized-table/statements.md
index ceda0c92364..1838845d708 100644
--- a/docs/content.zh/docs/dev/table/materialized-table/statements.md
+++ b/docs/content.zh/docs/dev/table/materialized-table/statements.md
@@ -96,7 +96,7 @@ CREATE MATERIALIZED TABLE my_materialized_table
 如上例所示,我们为 `ds` 分区列指定了 `date-formatter` 选项。每次调度时,调度时间将转换为相应的 `ds` 分区值。例如,在 
`2024-01-01 00:00:00` 的调度时间下,只有分区 `ds = '2024-01-01'` 会被刷新。
 
 注意
-- `partition.fields.#.date-formatter` 选项仅适用于全量模式。
+- [partition.fields.#.date-formatter]({{< ref "docs/dev/table/config" 
>}}#partition-fields-date-formatter) 选项仅适用于全量模式。
 - [partition.fields.#.date-formatter]({{< ref "docs/dev/table/config" 
>}}#partition-fields-date-formatter) 中的字段必须是有效的字符串类型分区字段。
 
 ## FRESHNESS
diff --git a/docs/content/docs/dev/table/materialized-table/overview.md 
b/docs/content/docs/dev/table/materialized-table/overview.md
index a2e6c830773..d49492e314a 100644
--- a/docs/content/docs/dev/table/materialized-table/overview.md
+++ b/docs/content/docs/dev/table/materialized-table/overview.md
@@ -45,8 +45,8 @@ Data freshness is a crucial attribute of a materialized 
table, serving two main
 - CONTINUOUS mode: Launches a Flink streaming job that continuously 
refreshes the materialized table data.
 - FULL mode: The workflow scheduler periodically triggers a Flink batch 
job to refresh the materialized table data.
 - **Determining the Refresh Frequency**.
-- In CONTINUOUS mode, data freshness is converted into the `checkpoint` 
interval of the Flink streaming job currently.
-- In FULL mode, data freshness is converted into the scheduling cycle of 
the workflow, e.g. cron expression.
+- In CONTINUOUS mode, data freshness is converted into the `checkpoint` 
interval of the Flink streaming job.
+- In FULL mode, data freshness is converted into the scheduling cycle of 
the workflow, e.g., a cron expression.
 
 ## Refresh Mode
 
diff --git a/docs/content/docs/dev/table/materialized-table/statements.md 
b/docs/content/docs/dev/table/materialized-table/statements.md
index 18d90675bb9..490652d0732 100644
--- a/docs/content/docs/dev/table/materialized-table/statements.md
+++ b/docs/content/docs/dev/table/materialized-table/statements.md
@@ -28,7 +28,7 @@ under the License.
 
 Flink SQL supports the following Materialized Table statements for now:
 - [CREATE MATERIALIZED TABLE](#create-materialized-table)
-- [Alter MATERIALIZED TABLE](#alter-materialized-table)
+- [ALTER MATERIALIZED TABLE](#alter-materialized-table)
 - [DROP MATERIALIZED TABLE](#drop-materialized-table)
 
 # CREATE MATERIALIZED TABLE
@@ -60,7 +60,7 @@ AS 
 
 ## PARTITIONED BY
 
-`PARTITIONED BY` define an optional list of columns to partition the 
materialized table. A directory is created for each partition if this 
materialized table is used as a filesystem sink.
+`PARTITIONED BY` defines an optional list of columns to partition the 
materialized table. A directory is created for each partition if this 
materialized table is used as a filesystem sink.
 
 **Example:**
 
@@ -96,12 +96,12 @@ CREATE MATERIALIZED TABLE my_materialized_table
 As shown in the above example, we specified the date-formatter option for the 
`ds` partition column. During each scheduling, the scheduling time will be 
converted to the ds partition value. For example, for a scheduling time of 
`2024-01-01 00:00:00`, only the partition `ds = '2024-01-01'` will be refreshed.
 
 Note
-- The `partition.fields.#.date-formatter` option only works in full mode.
+- The [partition.fields.#.date-formatter]({{< ref "docs/dev/table/config" 
>}}#partition-fields-date-formatter) option only works in full mode.
 - The field in the [partition.fields.#.date-formatter]({{< ref 
"docs/dev/table/config" >}}#partition-fields-date-formatter) must be a valid 
string type partition field.
 
 ## FRESHNESS
 
-`FRESHNESS` define the data freshness of a materialized table.
+`FRESHNESS` defines the data freshness of a materialized table.
 
 **FRESHNESS and Refresh Mode Relationship**
 



(flink) branch master updated (c00fcfb293d -> d04c70fae03)

2024-07-10 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from c00fcfb293d [FLINK-33730][doc] Update the Flink upgrade savepoint 
compatibility table doc for chinese version
 new 140d96c0041 [FLINK-35754][e2e] Fix 
SqlGatewayE2ECase.testMaterializedTableInFullMode failed due to Internal Server 
Error
 new d04c70fae03 [FLINK-35754][doc] Fix typo in materialized table docs

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../dev/table/materialized-table/statements.md |  2 +-
 .../docs/dev/table/materialized-table/overview.md  |  4 +--
 .../dev/table/materialized-table/statements.md |  8 +++---
 .../flink/table/gateway/SqlGatewayE2ECase.java | 13 +
 .../flink/core/testutils/CommonTestUtils.java  | 33 ++
 5 files changed, 47 insertions(+), 13 deletions(-)



(flink) 01/02: [FLINK-35754][e2e] Fix SqlGatewayE2ECase.testMaterializedTableInFullMode failed due to Internal Server Error

2024-07-10 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 140d96c00410d16d56f293b321e73475532878c6
Author: Feng Jin 
AuthorDate: Fri Jul 5 15:56:52 2024 +0800

[FLINK-35754][e2e] Fix SqlGatewayE2ECase.testMaterializedTableInFullMode 
failed due to Internal Server Error
---
 .../flink/table/gateway/SqlGatewayE2ECase.java | 13 +
 .../flink/core/testutils/CommonTestUtils.java  | 33 ++
 2 files changed, 40 insertions(+), 6 deletions(-)

diff --git 
a/flink-end-to-end-tests/flink-sql-gateway-test/src/test/java/org/apache/flink/table/gateway/SqlGatewayE2ECase.java
 
b/flink-end-to-end-tests/flink-sql-gateway-test/src/test/java/org/apache/flink/table/gateway/SqlGatewayE2ECase.java
index d65ed2f5f5d..0d7387b5803 100644
--- 
a/flink-end-to-end-tests/flink-sql-gateway-test/src/test/java/org/apache/flink/table/gateway/SqlGatewayE2ECase.java
+++ 
b/flink-end-to-end-tests/flink-sql-gateway-test/src/test/java/org/apache/flink/table/gateway/SqlGatewayE2ECase.java
@@ -187,7 +187,7 @@ public class SqlGatewayE2ECase extends TestLogger {
 gatewayRestClient.executeStatementWithResult("SET 
'execution.runtime-mode' = 'batch'");
 
 // verify the result
-CommonTestUtils.waitUtil(
+CommonTestUtils.waitUntilIgnoringExceptions(
 () -> {
 List result =
 gatewayRestClient.executeStatementWithResult(
@@ -219,7 +219,7 @@ public class SqlGatewayE2ECase extends TestLogger {
 "ALTER MATERIALIZED TABLE 
my_materialized_table_in_continuous_mode RESUME");
 
 // verify the result
-CommonTestUtils.waitUtil(
+CommonTestUtils.waitUntilIgnoringExceptions(
 () -> {
 List result =
 gatewayRestClient.executeStatementWithResult(
@@ -312,7 +312,7 @@ public class SqlGatewayE2ECase extends TestLogger {
 
 // verify the materialized table should auto refresh the today 
partition or tomorrow
 // partition
-CommonTestUtils.waitUtil(
+CommonTestUtils.waitUntilIgnoringExceptions(
 () -> {
 List result =
 gatewayRestClient.executeStatementWithResult(
@@ -345,9 +345,10 @@ public class SqlGatewayE2ECase extends TestLogger {
 gatewayRestClient.executeStatementWithResult(
 "ALTER MATERIALIZED TABLE 
my_materialized_table_in_full_mode RESUME");
 
-// wait until the materialized table is updated and verify only 
today or tomorrow data
+// wait until the materialized table is updated and verify only 
today or tomorrow
+// data
 // should be updated
-CommonTestUtils.waitUtil(
+CommonTestUtils.waitUntilIgnoringExceptions(
 () -> {
 List result =
 gatewayRestClient.executeStatementWithResult(
@@ -378,7 +379,7 @@ public class SqlGatewayE2ECase extends TestLogger {
 + "')");
 
 // verify the materialized table that all partitions are updated
-CommonTestUtils.waitUtil(
+CommonTestUtils.waitUntilIgnoringExceptions(
 () -> {
 List result =
 gatewayRestClient.executeStatementWithResult(
diff --git 
a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java
 
b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java
index 7466648a904..70e474e9e86 100644
--- 
a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java
+++ 
b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java
@@ -19,6 +19,8 @@
 package org.apache.flink.core.testutils;
 
 import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.BufferedWriter;
 import java.io.ByteArrayInputStream;
@@ -42,6 +44,8 @@ import static org.hamcrest.MatcherAssert.assertThat;
 /** This class contains reusable utility methods for unit tests. */
 public class CommonTestUtils {
 
+private static final Logger LOG = 
LoggerFactory.getLogger(CommonTestUtils.class);
+
 /**
  * Creates a copy of an object via Java Serialization.
  *
@@ -215,6 +219,35 @@ public class CommonTestUtils {
 }
 }
 
+/**
+ * Wait until the given condition is met or timeout, ignoring any 
exceptions thrown by the
+ * condition.
+ 

(flink) branch release-1.20 updated (2b3a47d98bf -> cab08d95a85)

2024-07-05 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git


from 2b3a47d98bf [FLINK-35731][runtime] Fix incorrect parallelism 
configured detection for Sink V2.
 new 4cc77557024 [FLINK-35643][table] Fix incorrect comment in 
AlterMaterializedTableResumeOperation
 new 9ebfdb6be19 [FLINK-35643][doc] Add materialized table statement doc
 new cab08d95a85 [FLINK-35645][doc] Add quickstart for materialized table

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 docs/content.zh/docs/dev/table/functions/_index.md |   4 +-
 .../{functions => materialized-table}/_index.md|   4 +-
 .../docs/dev/table/materialized-table/overview.md  |  66 
 .../dev/table/materialized-table/quickstart.md | 313 ++
 .../dev/table/materialized-table/statements.md | 348 +
 docs/content/docs/dev/table/functions/_index.md|   4 +-
 .../docs/dev/table/materialized-table}/_index.md   |   4 +-
 .../docs/dev/table/materialized-table/overview.md  |  65 
 .../dev/table/materialized-table/quickstart.md | 313 ++
 .../dev/table/materialized-table/statements.md | 342 
 .../static/fig/materialized-table-architecture.svg |  21 ++
 .../AlterMaterializedTableResumeOperation.java |   2 +-
 12 files changed, 1477 insertions(+), 9 deletions(-)
 copy docs/content.zh/docs/dev/table/{functions => 
materialized-table}/_index.md (96%)
 create mode 100644 
docs/content.zh/docs/dev/table/materialized-table/overview.md
 create mode 100644 
docs/content.zh/docs/dev/table/materialized-table/quickstart.md
 create mode 100644 
docs/content.zh/docs/dev/table/materialized-table/statements.md
 copy docs/{content.zh/docs/dev/table/functions => 
content/docs/dev/table/materialized-table}/_index.md (96%)
 create mode 100644 docs/content/docs/dev/table/materialized-table/overview.md
 create mode 100644 docs/content/docs/dev/table/materialized-table/quickstart.md
 create mode 100644 docs/content/docs/dev/table/materialized-table/statements.md
 create mode 100644 docs/static/fig/materialized-table-architecture.svg



(flink) 03/03: [FLINK-35645][doc] Add quickstart for materialized table

2024-07-05 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git

commit cab08d95a85c53d0c23d1fc3e116f772e6bf1765
Author: Feng Jin 
AuthorDate: Wed Jun 26 11:07:37 2024 +0800

[FLINK-35645][doc] Add quickstart for materialized table

This closes #24975

(cherry picked from commit 90fc679df073754b93eb5c220373daad7dca0a32)
---
 .../dev/table/materialized-table/quickstart.md | 313 +
 .../dev/table/materialized-table/quickstart.md | 313 +
 .../static/fig/materialized-table-architecture.svg |  21 ++
 3 files changed, 647 insertions(+)

diff --git a/docs/content.zh/docs/dev/table/materialized-table/quickstart.md 
b/docs/content.zh/docs/dev/table/materialized-table/quickstart.md
new file mode 100644
index 000..7b5f5b42abe
--- /dev/null
+++ b/docs/content.zh/docs/dev/table/materialized-table/quickstart.md
@@ -0,0 +1,313 @@
+---
+title: 快速入门
+weight: 3
+type: docs
+aliases:
+- /dev/table/materialized-table/quickstart.html
+---
+
+
+
+# 快速入门
+
+本入门指南将帮助你快速了解并开始使用物化表。内容包括环境设置,以及创建、修改和删除连续模式和全量模式的物化表。
+
+# 架构介绍
+
+- **Client**: 可以是任何能够与 [Flink SQL Gateway]({{< ref 
"docs/dev/table/sql-gateway/overview" >}}) 交互的客户端,如 [SQL 客户端]({{< ref 
"docs/dev/table/sqlClient" >}})、[Flink JDBC 驱动]({{< ref 
"docs/dev/table/jdbcDriver" >}}) 等。
+- **Flink SQL Gateway**: 支持创建、修改和删除物化表。并包含了一个内置的工作流调度器,用于定期刷新全量模式的物化表。
+- **Flink Cluster**: 用于运行物化表刷新作业的 Flink 集群。
+- **Catalog**: 负责管理物化表元数据的创建、查询、修改和删除。
+- **Catalog Store**: 提供 Catalog 属性持久化功能,以便在操作物化表时自动初始化 Catalog 并获取相关的元数据。
+
+{{< img src="/fig/materialized-table-architecture.svg" alt="Illustration of 
Flink Materialized Table Architecture" width="85%" >}}
+
+# 环境搭建
+
+## 目录准备
+
+**请将下面的示例路径替换为你机器上的实际路径。**
+
+- 创建 Catalog Store 和 Catalog 所需的目录
+
+```
+# 用于 File Catalog Store 保存 Catalog 属性
+mkdir -p {catalog_store_path}
+
+# 用于 test-filesystem Catalog 保存元数据和表数据
+mkdir -p {catalog_path}
+
+# 用于 test-filesystem Catalog 的默认数据库
+mkdir -p {catalog_path}/mydb
+```
+
+- 创建目录分别用于保存 Checkpoints 和 Savepoints:
+
+```
+mkdir -p {checkpoints_path}
+
+mkdir -p {savepoints_path}
+```
+
+## 资源准备
+
+这里的方法和[本地安装]({{< ref "docs/try-flink/local_installation" >}})中记录的步骤类似。Flink 
可以运行在任何类 UNIX 的操作系统下面,例如:Linux, Mac OS X 和 Cygwin (for Windows)。
+
+[下载](https://flink.apache.org/downloads/) Flink 最新的二进制包并进行解压:
+
+```
+tar -xzf flink-*.tgz
+```
+
+[下载](https://https://repo.maven.apache.org/maven2/org/apache/flink/flink-table-filesystem-test-utils/)
 test-filesystem 连接器, 并将其放入 lib 目录:
+
+```
+cp flink-table-filesystem-test-utils-{VERSION}.jar flink-*/lib/
+```
+
+## 配置准备
+
+在 `config.yaml` 文件中添加以下配置:
+
+```yaml
+execution:
+  checkpoints:
+dir: file://{checkpoints_path}
+
+# 配置 file catalog store
+table:
+  catalog-store:
+kind: file
+file:
+  path: {catalog_store_path}
+
+# 配置 embedded 调度器 
+workflow-scheduler:
+  type: embedded
+
+# 配置 SQL gateway 的地址和端口
+sql-gateway:
+  endpoint:
+rest:
+  address: 127.0.0.1
+  port: 8083
+```
+
+## 启动 Flink Cluster
+
+运行以下脚本,即可在本地启动集群:
+
+```
+./bin/start-cluster.sh
+```
+
+## 启动 SQL Gateway
+
+运行以下脚本,即可在本地启动 SQL Gateway:
+
+```
+./sql-gateway.sh start
+```
+
+## 启动 SQL Client
+
+运行以下脚本,即可在本地启动 SQL Client 客户端并连接到指定的 SQL Gateway:
+
+```
+./sql-client.sh gateway --endpoint http://127.0.0.1:8083
+```
+
+## 创建 Catalog 和 Source 表
+
+- 创建 test-filesystem catalog 用于后续创建物化表。
+
+```sql
+CREATE CATALOG mt_cat WITH (
+  'type' = 'test-filesystem',
+  'path' = '{catalog_path}',
+  'default-database' = 'mydb'
+);
+
+USE CATALOG mt_cat;
+```
+
+- 创建 Source 表作为物化表的数据源。
+
+```sql
+-- 1. 创建 Source 表,并指定 Source 表的数据格式为 json
+CREATE TABLE json_source (
+  order_id BIGINT,
+  user_id BIGINT,
+  user_name STRING,
+  order_created_at STRING,
+  payment_amount_cents BIGINT
+) WITH (
+  'format' = 'json',
+  'source.monitor-interval' = '10s'
+);
+
+-- 2. 插入一些测试数据
+INSERT INTO json_source VALUES 
+  (1001, 1, 'user1', '2024-06-19', 10),
+  (1002, 2, 'user2', '2024-06-19', 20),
+  (1003, 3, 'user3', '2024-06-19', 30),
+  (1004, 4, 'user4', '2024-06-19', 40),
+  (1005, 1, 'user1', '2024-06-20', 10),
+  (1006, 2, 'user2', '2024-06-20', 20),
+  (1007, 3, 'user3', '2024-06-20', 30),
+  (1008, 4, 'user4', '2024-06-20', 40);
+```
+
+# 创建连续模式物化表
+
+## 创建物化表
+
+创建一个连续模式的物化表,对应的数据新鲜度为 `30` 秒。通过 `http://localhost:8081` 页面可以查看对应的 Flink 
流作业,该作业处于 `RUNNING` 状态,对应的 `checkpoint` 间隔为 `30` 秒。
+
+```sql
+CREATE MATERIALIZED TABLE continuous_users_shops
+PARTITIONED BY (ds)
+WITH (
+  'format' = 'debezium-json',
+  'sink.rolling-p

(flink) 01/03: [FLINK-35643][table] Fix incorrect comment in AlterMaterializedTableResumeOperation

2024-07-05 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 4cc775570247da1c326eb341fae600d70d2e6c9a
Author: Feng Jin 
AuthorDate: Wed Jul 3 18:36:02 2024 +0800

[FLINK-35643][table] Fix incorrect comment in 
AlterMaterializedTableResumeOperation

(cherry picked from commit 2efeb20f8cd2fb258765cf5c50dd180e10f669ed)
---
 .../materializedtable/AlterMaterializedTableResumeOperation.java| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/AlterMaterializedTableResumeOperation.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/AlterMaterializedTableResumeOperation.java
index 907d5b7cc3b..0acfca464a0 100644
--- 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/AlterMaterializedTableResumeOperation.java
+++ 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/AlterMaterializedTableResumeOperation.java
@@ -25,7 +25,7 @@ import org.apache.flink.table.operations.OperationUtils;
 
 import java.util.Map;
 
-/** Operation to describe a ALTER MATERIALIZED TABLE ... SUSPEND statement. */
+/** Operation to describe a ALTER MATERIALIZED TABLE ... RESUME statement. */
 @Internal
 public class AlterMaterializedTableResumeOperation extends 
AlterMaterializedTableOperation {
 



(flink) 02/03: [FLINK-35643][doc] Add materialized table statement doc

2024-07-05 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 9ebfdb6be1947654a1ee6341b70e2fe10a2f5011
Author: Feng Jin 
AuthorDate: Sun Jun 23 20:43:59 2024 +0800

[FLINK-35643][doc] Add materialized table statement doc

This closes #24975

(cherry picked from commit 87be68a3dd6cac6e67d762a21d2e684c0573b958)
---
 docs/content.zh/docs/dev/table/functions/_index.md |   4 +-
 .../{functions => materialized-table}/_index.md|   4 +-
 .../docs/dev/table/materialized-table/overview.md  |  66 
 .../dev/table/materialized-table/statements.md | 348 +
 docs/content/docs/dev/table/functions/_index.md|   4 +-
 .../docs/dev/table/materialized-table}/_index.md   |   4 +-
 .../docs/dev/table/materialized-table/overview.md  |  65 
 .../dev/table/materialized-table/statements.md | 342 
 8 files changed, 829 insertions(+), 8 deletions(-)

diff --git a/docs/content.zh/docs/dev/table/functions/_index.md 
b/docs/content.zh/docs/dev/table/functions/_index.md
index 7e059a9a1c7..b01f9295fe5 100644
--- a/docs/content.zh/docs/dev/table/functions/_index.md
+++ b/docs/content.zh/docs/dev/table/functions/_index.md
@@ -1,7 +1,7 @@
 ---
 title: 函数
 bookCollapseSection: true
-weight: 33
+weight: 34
 ---
 
\ No newline at end of file
+-->
diff --git a/docs/content.zh/docs/dev/table/functions/_index.md 
b/docs/content.zh/docs/dev/table/materialized-table/_index.md
similarity index 96%
copy from docs/content.zh/docs/dev/table/functions/_index.md
copy to docs/content.zh/docs/dev/table/materialized-table/_index.md
index 7e059a9a1c7..9dbebca64bb 100644
--- a/docs/content.zh/docs/dev/table/functions/_index.md
+++ b/docs/content.zh/docs/dev/table/materialized-table/_index.md
@@ -1,5 +1,5 @@
 ---
-title: 函数
+title: Materialized Table
 bookCollapseSection: true
 weight: 33
 ---
@@ -20,4 +20,4 @@ software distributed under the License is distributed on an
 KIND, either express or implied.  See the License for the
 specific language governing permissions and limitations
 under the License.
--->
\ No newline at end of file
+-->
diff --git a/docs/content.zh/docs/dev/table/materialized-table/overview.md 
b/docs/content.zh/docs/dev/table/materialized-table/overview.md
new file mode 100644
index 000..da82ca0ebc9
--- /dev/null
+++ b/docs/content.zh/docs/dev/table/materialized-table/overview.md
@@ -0,0 +1,66 @@
+---
+title: 概览
+weight: 1
+type: docs
+aliases:
+- /dev/table/materialized-table/
+---
+
+
+# 介绍
+
+物化表是 Flink SQL 引入的一种新的表类型,旨在简化批处理和流处理数据管道,提供一致的开发体验。在创建物化表时,通过指定数据新鲜度和查询,Flink 
引擎会自动推导出物化表的 Schema ,并创建相应的数据刷新管道,以达到指定的新鲜度。
+
+{{< hint warning >}}
+**注意**:该功能目前是一个 MVP(最小可行产品)功能,仅在 [SQL Gateway]({{< ref 
"docs/dev/table/sql-gateway/overview" >}})中可用,并且只支持部署作业到 Flink [Standalone]({{< 
ref "docs/deployment/resource-providers/standalone/overview" >}})集群。
+{{< /hint >}}
+
+# 核心概念
+
+物化表包含以下核心概念:数据新鲜度、刷新模式、查询定义和 `Schema` 。
+
+## 数据新鲜度
+
+数据新鲜度定义了物化表数据相对于基础表更新的最大滞后时间。它并非绝对保证,而是 Flink 尝试达到的目标。框架会尽力确保物化表中的数据在指定的新鲜度内刷新。
+
+数据新鲜度是物化表的一个关键属性,具有两个主要作用:
+- **确定刷新模式**:目前有连续模式和全量模式。关于如何确定刷新模式的详细信息,请参阅 
[materialized-table.refresh-mode.freshness-threshold]({{< ref 
"docs/dev/table/config" 
>}}#materialized-table-refresh-mode-freshness-threshold) 配置项。
+- 连续模式:启动 Flink 流作业,持续刷新物化表数据。
+- 全量模式:工作流调度器定期触发 Flink 批处理作业,全量刷新物化表数据。
+- **确定刷新频率**:
+- 连续模式下,数据新鲜度转换为 Flink 流作业的 `checkpoint` 间隔。
+- 全量模式下,数据新鲜度转换为工作流的调度周期,例如 `cron` 表达式。
+
+## 刷新模式
+
+刷新模式有连续模式和全量模式两种。默认情况下,根据数据新鲜度推断刷新模式。用户可以为特定业务场景显式指定刷新模式,它的优先级高于根据数据新鲜度推导的刷新模式。
+
+- **连续模式**:Flink 流作业会增量更新物化表数据,下游数据会立即可见,或者等 checkpoint 完成时才可见,由对应的 Connector 
行为决定。
+- **全量模式**:调度器会定期触发对物化表数据的全量覆盖,其数据刷新周期与工作流的调度周期相匹配。
+- 默认的覆盖行为是表级别的。如果分区字段存在,并且通过 [partition.fields.#.date-formatter]({{< ref 
"docs/dev/table/config" >}}#partition-fields-date-formatter) 
指定了时间分区字段格式,则按照分区粒度覆盖,即每次只刷新最新的分区。
+
+## 查询定义
+
+物化表的查询定义支持所有 Flink SQL [查询]({{< ref "docs/dev/table/sql/queries/overview" 
>}})。查询结果用于填充物化表。在连续模式下,查询结果会持续更新到物化表中,而在全量模式下,每次查询结果都会覆盖更新到物化表。
+
+## Schema
+
+物化表的 `Schema` 定义与普通表相同,可以声明主键和分区字段。其列名和类型会从相应的查询中推导,用户无法手动指定。
+
diff --git a/docs/content.zh/docs/dev/table/materialized-table/statements.md 
b/docs/content.zh/docs/dev/table/materialized-table/statements.md
new file mode 100644
index 000..ceda0c92364
--- /dev/null
+++ b/docs/content.zh/docs/dev/table/materialized-table/statements.md
@@ -0,0 +1,348 @@
+---
+title: 语法说明
+weight: 2
+type: docs
+aliases:
+- /dev/table/materialized-table/statements.html
+---
+
+
+# 物化表语法
+
+Flink SQL 目前支持以下物化表操作:
+- [CREATE MATERIALIZED TABLE](#create-materialized-table)
+- [ALTER MATERIALIZED TABLE](#alter-materialized-table)
+- [DROP MATERIALIZED TABLE](#drop-materialized-table)
+
+# CREATE MATERIALIZED TABLE
+
+```
+CREATE MATERIALIZED TABLE [catalog_name.][db

(flink) branch master updated (59769b7eaa5 -> 90fc679df07)

2024-07-05 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 59769b7eaa5 [FLINK-35755] Upgrade to flink-shaded 19.0
 new 2efeb20f8cd [FLINK-35643][table] Fix incorrect comment in 
AlterMaterializedTableResumeOperation
 new 87be68a3dd6 [FLINK-35643][doc] Add materialized table statement doc
 new 90fc679df07 [FLINK-35645][doc] Add quickstart for materialized table

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 docs/content.zh/docs/dev/table/functions/_index.md |   4 +-
 .../{functions => materialized-table}/_index.md|   4 +-
 .../docs/dev/table/materialized-table/overview.md  |  66 
 .../dev/table/materialized-table/quickstart.md | 313 ++
 .../dev/table/materialized-table/statements.md | 348 +
 docs/content/docs/dev/table/functions/_index.md|   4 +-
 .../docs/dev/table/materialized-table}/_index.md   |   4 +-
 .../docs/dev/table/materialized-table/overview.md  |  65 
 .../dev/table/materialized-table/quickstart.md | 313 ++
 .../dev/table/materialized-table/statements.md | 342 
 .../static/fig/materialized-table-architecture.svg |  21 ++
 .../AlterMaterializedTableResumeOperation.java |   2 +-
 12 files changed, 1477 insertions(+), 9 deletions(-)
 copy docs/content.zh/docs/dev/table/{functions => 
materialized-table}/_index.md (96%)
 create mode 100644 
docs/content.zh/docs/dev/table/materialized-table/overview.md
 create mode 100644 
docs/content.zh/docs/dev/table/materialized-table/quickstart.md
 create mode 100644 
docs/content.zh/docs/dev/table/materialized-table/statements.md
 copy docs/{content.zh/docs/dev/table/functions => 
content/docs/dev/table/materialized-table}/_index.md (96%)
 create mode 100644 docs/content/docs/dev/table/materialized-table/overview.md
 create mode 100644 docs/content/docs/dev/table/materialized-table/quickstart.md
 create mode 100644 docs/content/docs/dev/table/materialized-table/statements.md
 create mode 100644 docs/static/fig/materialized-table-architecture.svg



(flink) 02/03: [FLINK-35643][doc] Add materialized table statement doc

2024-07-05 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 87be68a3dd6cac6e67d762a21d2e684c0573b958
Author: Feng Jin 
AuthorDate: Sun Jun 23 20:43:59 2024 +0800

[FLINK-35643][doc] Add materialized table statement doc

This closes #24975
---
 docs/content.zh/docs/dev/table/functions/_index.md |   4 +-
 .../{functions => materialized-table}/_index.md|   4 +-
 .../docs/dev/table/materialized-table/overview.md  |  66 
 .../dev/table/materialized-table/statements.md | 348 +
 docs/content/docs/dev/table/functions/_index.md|   4 +-
 .../docs/dev/table/materialized-table}/_index.md   |   4 +-
 .../docs/dev/table/materialized-table/overview.md  |  65 
 .../dev/table/materialized-table/statements.md | 342 
 8 files changed, 829 insertions(+), 8 deletions(-)

diff --git a/docs/content.zh/docs/dev/table/functions/_index.md 
b/docs/content.zh/docs/dev/table/functions/_index.md
index 7e059a9a1c7..b01f9295fe5 100644
--- a/docs/content.zh/docs/dev/table/functions/_index.md
+++ b/docs/content.zh/docs/dev/table/functions/_index.md
@@ -1,7 +1,7 @@
 ---
 title: 函数
 bookCollapseSection: true
-weight: 33
+weight: 34
 ---
 
\ No newline at end of file
+-->
diff --git a/docs/content.zh/docs/dev/table/functions/_index.md 
b/docs/content.zh/docs/dev/table/materialized-table/_index.md
similarity index 96%
copy from docs/content.zh/docs/dev/table/functions/_index.md
copy to docs/content.zh/docs/dev/table/materialized-table/_index.md
index 7e059a9a1c7..9dbebca64bb 100644
--- a/docs/content.zh/docs/dev/table/functions/_index.md
+++ b/docs/content.zh/docs/dev/table/materialized-table/_index.md
@@ -1,5 +1,5 @@
 ---
-title: 函数
+title: Materialized Table
 bookCollapseSection: true
 weight: 33
 ---
@@ -20,4 +20,4 @@ software distributed under the License is distributed on an
 KIND, either express or implied.  See the License for the
 specific language governing permissions and limitations
 under the License.
--->
\ No newline at end of file
+-->
diff --git a/docs/content.zh/docs/dev/table/materialized-table/overview.md 
b/docs/content.zh/docs/dev/table/materialized-table/overview.md
new file mode 100644
index 000..da82ca0ebc9
--- /dev/null
+++ b/docs/content.zh/docs/dev/table/materialized-table/overview.md
@@ -0,0 +1,66 @@
+---
+title: 概览
+weight: 1
+type: docs
+aliases:
+- /dev/table/materialized-table/
+---
+
+
+# 介绍
+
+物化表是 Flink SQL 引入的一种新的表类型,旨在简化批处理和流处理数据管道,提供一致的开发体验。在创建物化表时,通过指定数据新鲜度和查询,Flink 
引擎会自动推导出物化表的 Schema ,并创建相应的数据刷新管道,以达到指定的新鲜度。
+
+{{< hint warning >}}
+**注意**:该功能目前是一个 MVP(最小可行产品)功能,仅在 [SQL Gateway]({{< ref 
"docs/dev/table/sql-gateway/overview" >}})中可用,并且只支持部署作业到 Flink [Standalone]({{< 
ref "docs/deployment/resource-providers/standalone/overview" >}})集群。
+{{< /hint >}}
+
+# 核心概念
+
+物化表包含以下核心概念:数据新鲜度、刷新模式、查询定义和 `Schema` 。
+
+## 数据新鲜度
+
+数据新鲜度定义了物化表数据相对于基础表更新的最大滞后时间。它并非绝对保证,而是 Flink 尝试达到的目标。框架会尽力确保物化表中的数据在指定的新鲜度内刷新。
+
+数据新鲜度是物化表的一个关键属性,具有两个主要作用:
+- **确定刷新模式**:目前有连续模式和全量模式。关于如何确定刷新模式的详细信息,请参阅 
[materialized-table.refresh-mode.freshness-threshold]({{< ref 
"docs/dev/table/config" 
>}}#materialized-table-refresh-mode-freshness-threshold) 配置项。
+- 连续模式:启动 Flink 流作业,持续刷新物化表数据。
+- 全量模式:工作流调度器定期触发 Flink 批处理作业,全量刷新物化表数据。
+- **确定刷新频率**:
+- 连续模式下,数据新鲜度转换为 Flink 流作业的 `checkpoint` 间隔。
+- 全量模式下,数据新鲜度转换为工作流的调度周期,例如 `cron` 表达式。
+
+## 刷新模式
+
+刷新模式有连续模式和全量模式两种。默认情况下,根据数据新鲜度推断刷新模式。用户可以为特定业务场景显式指定刷新模式,它的优先级高于根据数据新鲜度推导的刷新模式。
+
+- **连续模式**:Flink 流作业会增量更新物化表数据,下游数据会立即可见,或者等 checkpoint 完成时才可见,由对应的 Connector 
行为决定。
+- **全量模式**:调度器会定期触发对物化表数据的全量覆盖,其数据刷新周期与工作流的调度周期相匹配。
+- 默认的覆盖行为是表级别的。如果分区字段存在,并且通过 [partition.fields.#.date-formatter]({{< ref 
"docs/dev/table/config" >}}#partition-fields-date-formatter) 
指定了时间分区字段格式,则按照分区粒度覆盖,即每次只刷新最新的分区。
+
+## 查询定义
+
+物化表的查询定义支持所有 Flink SQL [查询]({{< ref "docs/dev/table/sql/queries/overview" 
>}})。查询结果用于填充物化表。在连续模式下,查询结果会持续更新到物化表中,而在全量模式下,每次查询结果都会覆盖更新到物化表。
+
+## Schema
+
+物化表的 `Schema` 定义与普通表相同,可以声明主键和分区字段。其列名和类型会从相应的查询中推导,用户无法手动指定。
+
diff --git a/docs/content.zh/docs/dev/table/materialized-table/statements.md 
b/docs/content.zh/docs/dev/table/materialized-table/statements.md
new file mode 100644
index 000..ceda0c92364
--- /dev/null
+++ b/docs/content.zh/docs/dev/table/materialized-table/statements.md
@@ -0,0 +1,348 @@
+---
+title: 语法说明
+weight: 2
+type: docs
+aliases:
+- /dev/table/materialized-table/statements.html
+---
+
+
+# 物化表语法
+
+Flink SQL 目前支持以下物化表操作:
+- [CREATE MATERIALIZED TABLE](#create-materialized-table)
+- [ALTER MATERIALIZED TABLE](#alter-materialized-table)
+- [DROP MATERIALIZED TABLE](#drop-materialized-table)
+
+# CREATE MATERIALIZED TABLE
+
+```
+CREATE MATERIALIZED TABLE [catalog_name.][db_name.]table_name
+ 
+[ ([  ]) ]
+ 
+[COMMENT table_comment]
+ 
+[PARTITIONED BY (pa

(flink) 01/03: [FLINK-35643][table] Fix incorrect comment in AlterMaterializedTableResumeOperation

2024-07-05 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 2efeb20f8cd2fb258765cf5c50dd180e10f669ed
Author: Feng Jin 
AuthorDate: Wed Jul 3 18:36:02 2024 +0800

[FLINK-35643][table] Fix incorrect comment in 
AlterMaterializedTableResumeOperation
---
 .../materializedtable/AlterMaterializedTableResumeOperation.java| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/AlterMaterializedTableResumeOperation.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/AlterMaterializedTableResumeOperation.java
index 907d5b7cc3b..0acfca464a0 100644
--- 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/AlterMaterializedTableResumeOperation.java
+++ 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/AlterMaterializedTableResumeOperation.java
@@ -25,7 +25,7 @@ import org.apache.flink.table.operations.OperationUtils;
 
 import java.util.Map;
 
-/** Operation to describe a ALTER MATERIALIZED TABLE ... SUSPEND statement. */
+/** Operation to describe a ALTER MATERIALIZED TABLE ... RESUME statement. */
 @Internal
 public class AlterMaterializedTableResumeOperation extends 
AlterMaterializedTableOperation {
 



(flink) 03/03: [FLINK-35645][doc] Add quickstart for materialized table

2024-07-05 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 90fc679df073754b93eb5c220373daad7dca0a32
Author: Feng Jin 
AuthorDate: Wed Jun 26 11:07:37 2024 +0800

[FLINK-35645][doc] Add quickstart for materialized table

This closes #24975
---
 .../dev/table/materialized-table/quickstart.md | 313 +
 .../dev/table/materialized-table/quickstart.md | 313 +
 .../static/fig/materialized-table-architecture.svg |  21 ++
 3 files changed, 647 insertions(+)

diff --git a/docs/content.zh/docs/dev/table/materialized-table/quickstart.md 
b/docs/content.zh/docs/dev/table/materialized-table/quickstart.md
new file mode 100644
index 000..7b5f5b42abe
--- /dev/null
+++ b/docs/content.zh/docs/dev/table/materialized-table/quickstart.md
@@ -0,0 +1,313 @@
+---
+title: 快速入门
+weight: 3
+type: docs
+aliases:
+- /dev/table/materialized-table/quickstart.html
+---
+
+
+
+# 快速入门
+
+本入门指南将帮助你快速了解并开始使用物化表。内容包括环境设置,以及创建、修改和删除连续模式和全量模式的物化表。
+
+# 架构介绍
+
+- **Client**: 可以是任何能够与 [Flink SQL Gateway]({{< ref 
"docs/dev/table/sql-gateway/overview" >}}) 交互的客户端,如 [SQL 客户端]({{< ref 
"docs/dev/table/sqlClient" >}})、[Flink JDBC 驱动]({{< ref 
"docs/dev/table/jdbcDriver" >}}) 等。
+- **Flink SQL Gateway**: 支持创建、修改和删除物化表。并包含了一个内置的工作流调度器,用于定期刷新全量模式的物化表。
+- **Flink Cluster**: 用于运行物化表刷新作业的 Flink 集群。
+- **Catalog**: 负责管理物化表元数据的创建、查询、修改和删除。
+- **Catalog Store**: 提供 Catalog 属性持久化功能,以便在操作物化表时自动初始化 Catalog 并获取相关的元数据。
+
+{{< img src="/fig/materialized-table-architecture.svg" alt="Illustration of 
Flink Materialized Table Architecture" width="85%" >}}
+
+# 环境搭建
+
+## 目录准备
+
+**请将下面的示例路径替换为你机器上的实际路径。**
+
+- 创建 Catalog Store 和 Catalog 所需的目录
+
+```
+# 用于 File Catalog Store 保存 Catalog 属性
+mkdir -p {catalog_store_path}
+
+# 用于 test-filesystem Catalog 保存元数据和表数据
+mkdir -p {catalog_path}
+
+# 用于 test-filesystem Catalog 的默认数据库
+mkdir -p {catalog_path}/mydb
+```
+
+- 创建目录分别用于保存 Checkpoints 和 Savepoints:
+
+```
+mkdir -p {checkpoints_path}
+
+mkdir -p {savepoints_path}
+```
+
+## 资源准备
+
+这里的方法和[本地安装]({{< ref "docs/try-flink/local_installation" >}})中记录的步骤类似。Flink 
可以运行在任何类 UNIX 的操作系统下面,例如:Linux, Mac OS X 和 Cygwin (for Windows)。
+
+[下载](https://flink.apache.org/downloads/) Flink 最新的二进制包并进行解压:
+
+```
+tar -xzf flink-*.tgz
+```
+
+[下载](https://https://repo.maven.apache.org/maven2/org/apache/flink/flink-table-filesystem-test-utils/)
 test-filesystem 连接器, 并将其放入 lib 目录:
+
+```
+cp flink-table-filesystem-test-utils-{VERSION}.jar flink-*/lib/
+```
+
+## 配置准备
+
+在 `config.yaml` 文件中添加以下配置:
+
+```yaml
+execution:
+  checkpoints:
+dir: file://{checkpoints_path}
+
+# 配置 file catalog store
+table:
+  catalog-store:
+kind: file
+file:
+  path: {catalog_store_path}
+
+# 配置 embedded 调度器 
+workflow-scheduler:
+  type: embedded
+
+# 配置 SQL gateway 的地址和端口
+sql-gateway:
+  endpoint:
+rest:
+  address: 127.0.0.1
+  port: 8083
+```
+
+## 启动 Flink Cluster
+
+运行以下脚本,即可在本地启动集群:
+
+```
+./bin/start-cluster.sh
+```
+
+## 启动 SQL Gateway
+
+运行以下脚本,即可在本地启动 SQL Gateway:
+
+```
+./sql-gateway.sh start
+```
+
+## 启动 SQL Client
+
+运行以下脚本,即可在本地启动 SQL Client 客户端并连接到指定的 SQL Gateway:
+
+```
+./sql-client.sh gateway --endpoint http://127.0.0.1:8083
+```
+
+## 创建 Catalog 和 Source 表
+
+- 创建 test-filesystem catalog 用于后续创建物化表。
+
+```sql
+CREATE CATALOG mt_cat WITH (
+  'type' = 'test-filesystem',
+  'path' = '{catalog_path}',
+  'default-database' = 'mydb'
+);
+
+USE CATALOG mt_cat;
+```
+
+- 创建 Source 表作为物化表的数据源。
+
+```sql
+-- 1. 创建 Source 表,并指定 Source 表的数据格式为 json
+CREATE TABLE json_source (
+  order_id BIGINT,
+  user_id BIGINT,
+  user_name STRING,
+  order_created_at STRING,
+  payment_amount_cents BIGINT
+) WITH (
+  'format' = 'json',
+  'source.monitor-interval' = '10s'
+);
+
+-- 2. 插入一些测试数据
+INSERT INTO json_source VALUES 
+  (1001, 1, 'user1', '2024-06-19', 10),
+  (1002, 2, 'user2', '2024-06-19', 20),
+  (1003, 3, 'user3', '2024-06-19', 30),
+  (1004, 4, 'user4', '2024-06-19', 40),
+  (1005, 1, 'user1', '2024-06-20', 10),
+  (1006, 2, 'user2', '2024-06-20', 20),
+  (1007, 3, 'user3', '2024-06-20', 30),
+  (1008, 4, 'user4', '2024-06-20', 40);
+```
+
+# 创建连续模式物化表
+
+## 创建物化表
+
+创建一个连续模式的物化表,对应的数据新鲜度为 `30` 秒。通过 `http://localhost:8081` 页面可以查看对应的 Flink 
流作业,该作业处于 `RUNNING` 状态,对应的 `checkpoint` 间隔为 `30` 秒。
+
+```sql
+CREATE MATERIALIZED TABLE continuous_users_shops
+PARTITIONED BY (ds)
+WITH (
+  'format' = 'debezium-json',
+  'sink.rolling-policy.rollover-interval' = '10s',
+  'sink.rolling-policy.ch

(flink) 01/02: [FLINK-35734][table] Fix periodic refresh job naming

2024-07-01 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git

commit cb3e9f4adfa602de972464555022a0d642d6b9cf
Author: Feng Jin 
AuthorDate: Mon Jul 1 16:33:39 2024 +0800

[FLINK-35734][table] Fix periodic refresh job naming
---
 .../service/materializedtable/MaterializedTableManager.java   | 11 ---
 1 file changed, 8 insertions(+), 3 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
index 293634e66d1..2a068187ad5 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
@@ -621,9 +621,14 @@ public class MaterializedTableManager {
 // Set job name, runtime mode
 Configuration customConfig = new Configuration();
 String jobName =
-String.format(
-"Materialized_table_%s_one_time_refresh_job",
-materializedTableIdentifier.asSerializableString());
+isPeriodic
+? String.format(
+"Materialized_table_%s_periodic_refresh_job",
+
materializedTableIdentifier.asSerializableString())
+: String.format(
+"Materialized_table_%s_one_time_refresh_job",
+
materializedTableIdentifier.asSerializableString());
+
 customConfig.set(NAME, jobName);
 customConfig.set(RUNTIME_MODE, BATCH);
 



(flink) branch release-1.20 updated (7f40c35fc74 -> d147bbf33d2)

2024-07-01 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git


from 7f40c35fc74 [FLINK-35699][k8s] Fix shading Jackson through fabric8
 new cb3e9f4adfa [FLINK-35734][table] Fix periodic refresh job naming
 new d147bbf33d2 [FLINK-35734][table] Do not override the user-defined 
checkpoint interval in continuous mode

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../MaterializedTableManager.java  | 24 ++---
 .../service/MaterializedTableStatementITCase.java  | 62 ++
 2 files changed, 80 insertions(+), 6 deletions(-)



(flink) 02/02: [FLINK-35734][table] Do not override the user-defined checkpoint interval in continuous mode

2024-07-01 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git

commit d147bbf33d2804092081b24ed7a43f244dc9580b
Author: Feng Jin 
AuthorDate: Mon Jul 1 16:54:17 2024 +0800

[FLINK-35734][table] Do not override the user-defined checkpoint interval 
in continuous mode
---
 .../MaterializedTableManager.java  | 13 +++--
 .../service/MaterializedTableStatementITCase.java  | 62 ++
 2 files changed, 72 insertions(+), 3 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
index 2a068187ad5..aa0877265a0 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
@@ -541,11 +541,18 @@ public class MaterializedTableManager {
 materializedTableIdentifier.asSerializableString());
 customConfig.set(NAME, jobName);
 customConfig.set(RUNTIME_MODE, STREAMING);
-customConfig.set(
-CheckpointingOptions.CHECKPOINTING_INTERVAL,
-catalogMaterializedTable.getFreshness());
 restorePath.ifPresent(s -> customConfig.set(SAVEPOINT_PATH, s));
 
+// Do not override the user-defined checkpoint interval
+if (!operationExecutor
+.getSessionContext()
+.getSessionConf()
+.contains(CheckpointingOptions.CHECKPOINTING_INTERVAL)) {
+customConfig.set(
+CheckpointingOptions.CHECKPOINTING_INTERVAL,
+catalogMaterializedTable.getFreshness());
+}
+
 String insertStatement =
 getInsertStatement(
 materializedTableIdentifier,
diff --git 
a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
 
b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
index 6ed91b954af..ef67d803246 100644
--- 
a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
+++ 
b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
@@ -73,6 +73,7 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 
+import static 
org.apache.flink.configuration.CheckpointingOptions.CHECKPOINTING_INTERVAL;
 import static 
org.apache.flink.table.api.config.TableConfigOptions.RESOURCES_DOWNLOAD_DIR;
 import static 
org.apache.flink.table.factories.FactoryUtil.WORKFLOW_SCHEDULER_TYPE;
 import static 
org.apache.flink.table.gateway.service.utils.SqlGatewayServiceTestUtil.awaitOperationTermination;
@@ -170,6 +171,67 @@ public class MaterializedTableStatementITCase extends 
AbstractMaterializedTableS
 assertThat(checkpointInterval).isEqualTo(30 * 1000);
 }
 
+@Test
+void 
testCreateMaterializedTableInContinuousModeWithCustomCheckpointInterval()
+throws Exception {
+
+// set checkpoint interval to 60 seconds
+long checkpointInterval = 60 * 1000;
+
+OperationHandle checkpointSetHandle =
+service.executeStatement(
+sessionHandle,
+String.format(
+"SET '%s' = '%d'",
+CHECKPOINTING_INTERVAL.key(), 
checkpointInterval),
+-1,
+new Configuration());
+awaitOperationTermination(service, sessionHandle, checkpointSetHandle);
+
+String materializedTableDDL =
+"CREATE MATERIALIZED TABLE users_shops"
++ " PARTITIONED BY (ds)\n"
++ " WITH(\n"
++ "   'format' = 'debezium-json'\n"
++ " )\n"
++ " FRESHNESS = INTERVAL '30' SECOND\n"
++ " AS SELECT \n"
++ "  user_id,\n"
++ "  shop_id,\n"
++ "  ds,\n"
++ "  SUM (payment_amount_cents) AS 
payed_buy_fee_sum,\n"
++ "  SUM (1) AS pv\n"
++ " FROM (\n"
++ "   

(flink) branch master updated (e7d7db3b6f8 -> 3aa3c48ab19)

2024-07-01 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from e7d7db3b6f8 [hotfix][parquet] Fix row group filter support issue 
(#24979)
 new 2cc90642166 [FLINK-35734][table] Fix periodic refresh job naming
 new 3aa3c48ab19 [FLINK-35734][table] Do not override the user-defined 
checkpoint interval in continuous mode

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../MaterializedTableManager.java  | 24 ++---
 .../service/MaterializedTableStatementITCase.java  | 62 ++
 2 files changed, 80 insertions(+), 6 deletions(-)



(flink) 01/02: [FLINK-35734][table] Fix periodic refresh job naming

2024-07-01 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 2cc90642166e9bb737b8a74a6c66e4c25d448908
Author: Feng Jin 
AuthorDate: Mon Jul 1 16:33:39 2024 +0800

[FLINK-35734][table] Fix periodic refresh job naming
---
 .../service/materializedtable/MaterializedTableManager.java   | 11 ---
 1 file changed, 8 insertions(+), 3 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
index 293634e66d1..2a068187ad5 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
@@ -621,9 +621,14 @@ public class MaterializedTableManager {
 // Set job name, runtime mode
 Configuration customConfig = new Configuration();
 String jobName =
-String.format(
-"Materialized_table_%s_one_time_refresh_job",
-materializedTableIdentifier.asSerializableString());
+isPeriodic
+? String.format(
+"Materialized_table_%s_periodic_refresh_job",
+
materializedTableIdentifier.asSerializableString())
+: String.format(
+"Materialized_table_%s_one_time_refresh_job",
+
materializedTableIdentifier.asSerializableString());
+
 customConfig.set(NAME, jobName);
 customConfig.set(RUNTIME_MODE, BATCH);
 



(flink) 02/02: [FLINK-35734][table] Do not override the user-defined checkpoint interval in continuous mode

2024-07-01 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 3aa3c48ab19360ec3055b3ae791d64e7b6d1ed3b
Author: Feng Jin 
AuthorDate: Mon Jul 1 16:54:17 2024 +0800

[FLINK-35734][table] Do not override the user-defined checkpoint interval 
in continuous mode
---
 .../MaterializedTableManager.java  | 13 +++--
 .../service/MaterializedTableStatementITCase.java  | 62 ++
 2 files changed, 72 insertions(+), 3 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
index 2a068187ad5..aa0877265a0 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
@@ -541,11 +541,18 @@ public class MaterializedTableManager {
 materializedTableIdentifier.asSerializableString());
 customConfig.set(NAME, jobName);
 customConfig.set(RUNTIME_MODE, STREAMING);
-customConfig.set(
-CheckpointingOptions.CHECKPOINTING_INTERVAL,
-catalogMaterializedTable.getFreshness());
 restorePath.ifPresent(s -> customConfig.set(SAVEPOINT_PATH, s));
 
+// Do not override the user-defined checkpoint interval
+if (!operationExecutor
+.getSessionContext()
+.getSessionConf()
+.contains(CheckpointingOptions.CHECKPOINTING_INTERVAL)) {
+customConfig.set(
+CheckpointingOptions.CHECKPOINTING_INTERVAL,
+catalogMaterializedTable.getFreshness());
+}
+
 String insertStatement =
 getInsertStatement(
 materializedTableIdentifier,
diff --git 
a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
 
b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
index 6ed91b954af..ef67d803246 100644
--- 
a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
+++ 
b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
@@ -73,6 +73,7 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 
+import static 
org.apache.flink.configuration.CheckpointingOptions.CHECKPOINTING_INTERVAL;
 import static 
org.apache.flink.table.api.config.TableConfigOptions.RESOURCES_DOWNLOAD_DIR;
 import static 
org.apache.flink.table.factories.FactoryUtil.WORKFLOW_SCHEDULER_TYPE;
 import static 
org.apache.flink.table.gateway.service.utils.SqlGatewayServiceTestUtil.awaitOperationTermination;
@@ -170,6 +171,67 @@ public class MaterializedTableStatementITCase extends 
AbstractMaterializedTableS
 assertThat(checkpointInterval).isEqualTo(30 * 1000);
 }
 
+@Test
+void 
testCreateMaterializedTableInContinuousModeWithCustomCheckpointInterval()
+throws Exception {
+
+// set checkpoint interval to 60 seconds
+long checkpointInterval = 60 * 1000;
+
+OperationHandle checkpointSetHandle =
+service.executeStatement(
+sessionHandle,
+String.format(
+"SET '%s' = '%d'",
+CHECKPOINTING_INTERVAL.key(), 
checkpointInterval),
+-1,
+new Configuration());
+awaitOperationTermination(service, sessionHandle, checkpointSetHandle);
+
+String materializedTableDDL =
+"CREATE MATERIALIZED TABLE users_shops"
++ " PARTITIONED BY (ds)\n"
++ " WITH(\n"
++ "   'format' = 'debezium-json'\n"
++ " )\n"
++ " FRESHNESS = INTERVAL '30' SECOND\n"
++ " AS SELECT \n"
++ "  user_id,\n"
++ "  shop_id,\n"
++ "  ds,\n"
++ "  SUM (payment_amount_cents) AS 
payed_buy_fee_sum,\n"
++ "  SUM (1) AS pv\n"
++ " FROM (\n"
++ "   

(flink) branch release-1.20 updated: [FLINK-35271][docs] Add doc for DESCRIBE JOB statement

2024-06-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.20 by this push:
 new cae9db9d523 [FLINK-35271][docs] Add doc for DESCRIBE JOB  
statement
cae9db9d523 is described below

commit cae9db9d523ca321061f097016fe1563d3cab8fc
Author: Xuyang 
AuthorDate: Thu Jun 27 10:13:21 2024 +0800

[FLINK-35271][docs] Add doc for DESCRIBE JOB  statement

This closes #24984

(cherry picked from commit 7fc3aac774f5deb9b48727ba5f916c78085b49b9)
---
 docs/content.zh/docs/dev/table/sql/describe.md | 35 +-
 docs/content.zh/docs/dev/table/sql/job.md  | 18 +
 docs/content/docs/dev/table/sql/describe.md| 34 -
 docs/content/docs/dev/table/sql/job.md | 19 ++
 4 files changed, 94 insertions(+), 12 deletions(-)

diff --git a/docs/content.zh/docs/dev/table/sql/describe.md 
b/docs/content.zh/docs/dev/table/sql/describe.md
index 4f77e3010ba..5f7480da1b5 100644
--- a/docs/content.zh/docs/dev/table/sql/describe.md
+++ b/docs/content.zh/docs/dev/table/sql/describe.md
@@ -28,7 +28,7 @@ under the License.
 
 # DESCRIBE 语句
 
-DESCRIBE 语句用于描述表或视图的 schema 或 catalog 的元数据。
+DESCRIBE 语句用于描述表或视图的 schema,或catalog 的元数据,或 Flink 集群上的指定作业。
 
 
 
@@ -36,17 +36,17 @@ DESCRIBE 语句用于描述表或视图的 schema 或 catalog 的元数据。
 
 {{< tabs "describe" >}}
 {{< tab "Java" >}}
-可以使用 `TableEnvironment` 的 `executeSql()` 方法执行 DESCRIBE 语句。如果 DESCRIBE 
操作执行成功,`executeSql()` 方法会返回给定表的 schema,否则会抛出异常。
+可以使用 `TableEnvironment` 的 `executeSql()` 方法执行 DESCRIBE 语句。如果 DESCRIBE 
操作执行成功,`executeSql()` 方法会返回所有对象,否则会抛出异常。
 
 以下示例展示了如何在 `TableEnvironment` 中执行一条 DESCRIBE 语句。
 {{< /tab >}}
 {{< tab "Scala" >}}
-可以使用 `TableEnvironment` 的 `executeSql()` 方法执行 DESCRIBE 语句。如果 DESCRIBE 
操作执行成功,`executeSql()` 方法会返回给定表的 schema,否则会抛出异常。
+可以使用 `TableEnvironment` 的 `executeSql()` 方法执行 DESCRIBE 语句。如果 DESCRIBE 
操作执行成功,`executeSql()` 方法会返回所有对象,否则会抛出异常。
 
 以下示例展示了如何在 `TableEnvironment` 中执行一条 DESCRIBE 语句。
 {{< /tab >}}
 {{< tab "Python" >}}
-可以使用 `TableEnvironment` 的 `execute_sql()` 方法执行 DESCRIBE 语句。如果 DESCRIBE 
操作执行成功,`execute_sql()` 方法会返回给定表的 schema,否则会抛出异常。
+可以使用 `TableEnvironment` 的 `execute_sql()` 方法执行 DESCRIBE 语句。如果 DESCRIBE 
操作执行成功,`execute_sql()` 方法会返回所有对象,否则会抛出异常。
 
 以下示例展示了如何在 `TableEnvironment` 中执行一条 DESCRIBE 语句。
 {{< /tab >}}
@@ -181,6 +181,10 @@ Flink SQL> CREATE CATALOG cat2 WITH 
('type'='generic_in_memory', 'default-databa
 Flink SQL> DESCRIBE CATALOG cat2;
 
 Flink SQL> DESC CATALOG EXTENDED cat2;
+  
+Flink SQL> DESCRIBE JOB '228d70913eab60dda85c5e7f78b5782c';
+  
+Flink SQL> DESC JOB '228d70913eab60dda85c5e7f78b5782c';
 ```
 {{< /tab >}}
 {{< /tabs >}}
@@ -329,6 +333,14 @@ Flink SQL> DESC CATALOG EXTENDED cat2;
 | option:default-database |db |
 +-+---+
 4 rows in set
+
+# DESCRIBE JOB '228d70913eab60dda85c5e7f78b5782c'
++--+--+-+-+
+|   job id | job name |  status |  start 
time |
++--+--+-+-+
+| 228d70913eab60dda85c5e7f78b5782c |myjob | RUNNING | 
2023-02-11T05:03:51.523 |
++--+--+-+-+
+1 row in set
 ```
 {{< /tab >}}
 {{< /tabs >}}
@@ -339,11 +351,22 @@ Flink SQL> DESC CATALOG EXTENDED cat2;
 
 ## 语法
 
-- DESCRIBE TABLE
+### DESCRIBE TABLE
+
 ```sql
 { DESCRIBE | DESC } [catalog_name.][db_name.]table_name
 ```
-- DESCRIBE CATALOG
+
+### DESCRIBE CATALOG
+
 ```sql
 { DESCRIBE | DESC } CATALOG [EXTENDED] catalog_name
 ```
+
+### DESCRIBE JOB
+
+```sql
+{ DESCRIBE | DESC } JOB ''
+```
+
+Attention DESCRIBE JOB 语句仅适用于 [SQL 
CLI]({{< ref "docs/dev/table/sqlClient" >}}) 或者 [SQL Gateway]({{< ref 
"docs/dev/table/sql-gateway/overview" >}}).
\ No newline at end of file
diff --git a/docs/content.zh/docs/dev/table/sql/job.md 
b/docs/content.zh/docs/dev/table/sql/job.md
index 5661969f4f2..eafb125bcc3 100644
--- a/docs/content.zh/docs/dev/table/sql/job.md
+++ b/docs/content.zh/docs/dev/table/sql/job.md
@@ -30,6 +30,7 @@ Job 语句用于管理作业的生命周期。
 
 目前 Flink SQL 支持以下 JOB 语句:
 - SHOW JOBS
+- DESCRIBE JOB
 - STOP JOB
 
 ## 执行 JOB 语句
@@ -52,6 +53,13 @@ Flink SQL> SHOW JOBS;
 | 228d70913eab60dda85c5e7f78b5782c |myjob | RUNNING | 
2023-02-11T05:03:51.523 |
 
+--+--+-+-+
 
+Flink SQL> DESCRIBE JOB '228d70913eab60dda85c5e7f78b5782c';
++--+--+-+--

(flink) branch master updated: [FLINK-35271][docs] Add doc for DESCRIBE JOB statement

2024-06-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 7fc3aac774f [FLINK-35271][docs] Add doc for DESCRIBE JOB  
statement
7fc3aac774f is described below

commit 7fc3aac774f5deb9b48727ba5f916c78085b49b9
Author: Xuyang 
AuthorDate: Thu Jun 27 10:13:21 2024 +0800

[FLINK-35271][docs] Add doc for DESCRIBE JOB  statement

This closes #24984
---
 docs/content.zh/docs/dev/table/sql/describe.md | 35 +-
 docs/content.zh/docs/dev/table/sql/job.md  | 18 +
 docs/content/docs/dev/table/sql/describe.md| 34 -
 docs/content/docs/dev/table/sql/job.md | 19 ++
 4 files changed, 94 insertions(+), 12 deletions(-)

diff --git a/docs/content.zh/docs/dev/table/sql/describe.md 
b/docs/content.zh/docs/dev/table/sql/describe.md
index 4f77e3010ba..5f7480da1b5 100644
--- a/docs/content.zh/docs/dev/table/sql/describe.md
+++ b/docs/content.zh/docs/dev/table/sql/describe.md
@@ -28,7 +28,7 @@ under the License.
 
 # DESCRIBE 语句
 
-DESCRIBE 语句用于描述表或视图的 schema 或 catalog 的元数据。
+DESCRIBE 语句用于描述表或视图的 schema,或catalog 的元数据,或 Flink 集群上的指定作业。
 
 
 
@@ -36,17 +36,17 @@ DESCRIBE 语句用于描述表或视图的 schema 或 catalog 的元数据。
 
 {{< tabs "describe" >}}
 {{< tab "Java" >}}
-可以使用 `TableEnvironment` 的 `executeSql()` 方法执行 DESCRIBE 语句。如果 DESCRIBE 
操作执行成功,`executeSql()` 方法会返回给定表的 schema,否则会抛出异常。
+可以使用 `TableEnvironment` 的 `executeSql()` 方法执行 DESCRIBE 语句。如果 DESCRIBE 
操作执行成功,`executeSql()` 方法会返回所有对象,否则会抛出异常。
 
 以下示例展示了如何在 `TableEnvironment` 中执行一条 DESCRIBE 语句。
 {{< /tab >}}
 {{< tab "Scala" >}}
-可以使用 `TableEnvironment` 的 `executeSql()` 方法执行 DESCRIBE 语句。如果 DESCRIBE 
操作执行成功,`executeSql()` 方法会返回给定表的 schema,否则会抛出异常。
+可以使用 `TableEnvironment` 的 `executeSql()` 方法执行 DESCRIBE 语句。如果 DESCRIBE 
操作执行成功,`executeSql()` 方法会返回所有对象,否则会抛出异常。
 
 以下示例展示了如何在 `TableEnvironment` 中执行一条 DESCRIBE 语句。
 {{< /tab >}}
 {{< tab "Python" >}}
-可以使用 `TableEnvironment` 的 `execute_sql()` 方法执行 DESCRIBE 语句。如果 DESCRIBE 
操作执行成功,`execute_sql()` 方法会返回给定表的 schema,否则会抛出异常。
+可以使用 `TableEnvironment` 的 `execute_sql()` 方法执行 DESCRIBE 语句。如果 DESCRIBE 
操作执行成功,`execute_sql()` 方法会返回所有对象,否则会抛出异常。
 
 以下示例展示了如何在 `TableEnvironment` 中执行一条 DESCRIBE 语句。
 {{< /tab >}}
@@ -181,6 +181,10 @@ Flink SQL> CREATE CATALOG cat2 WITH 
('type'='generic_in_memory', 'default-databa
 Flink SQL> DESCRIBE CATALOG cat2;
 
 Flink SQL> DESC CATALOG EXTENDED cat2;
+  
+Flink SQL> DESCRIBE JOB '228d70913eab60dda85c5e7f78b5782c';
+  
+Flink SQL> DESC JOB '228d70913eab60dda85c5e7f78b5782c';
 ```
 {{< /tab >}}
 {{< /tabs >}}
@@ -329,6 +333,14 @@ Flink SQL> DESC CATALOG EXTENDED cat2;
 | option:default-database |db |
 +-+---+
 4 rows in set
+
+# DESCRIBE JOB '228d70913eab60dda85c5e7f78b5782c'
++--+--+-+-+
+|   job id | job name |  status |  start 
time |
++--+--+-+-+
+| 228d70913eab60dda85c5e7f78b5782c |myjob | RUNNING | 
2023-02-11T05:03:51.523 |
++--+--+-+-+
+1 row in set
 ```
 {{< /tab >}}
 {{< /tabs >}}
@@ -339,11 +351,22 @@ Flink SQL> DESC CATALOG EXTENDED cat2;
 
 ## 语法
 
-- DESCRIBE TABLE
+### DESCRIBE TABLE
+
 ```sql
 { DESCRIBE | DESC } [catalog_name.][db_name.]table_name
 ```
-- DESCRIBE CATALOG
+
+### DESCRIBE CATALOG
+
 ```sql
 { DESCRIBE | DESC } CATALOG [EXTENDED] catalog_name
 ```
+
+### DESCRIBE JOB
+
+```sql
+{ DESCRIBE | DESC } JOB ''
+```
+
+Attention DESCRIBE JOB 语句仅适用于 [SQL 
CLI]({{< ref "docs/dev/table/sqlClient" >}}) 或者 [SQL Gateway]({{< ref 
"docs/dev/table/sql-gateway/overview" >}}).
\ No newline at end of file
diff --git a/docs/content.zh/docs/dev/table/sql/job.md 
b/docs/content.zh/docs/dev/table/sql/job.md
index 5661969f4f2..eafb125bcc3 100644
--- a/docs/content.zh/docs/dev/table/sql/job.md
+++ b/docs/content.zh/docs/dev/table/sql/job.md
@@ -30,6 +30,7 @@ Job 语句用于管理作业的生命周期。
 
 目前 Flink SQL 支持以下 JOB 语句:
 - SHOW JOBS
+- DESCRIBE JOB
 - STOP JOB
 
 ## 执行 JOB 语句
@@ -52,6 +53,13 @@ Flink SQL> SHOW JOBS;
 | 228d70913eab60dda85c5e7f78b5782c |myjob | RUNNING | 
2023-02-11T05:03:51.523 |
 
+--+--+-+-+
 
+Flink SQL> DESCRIBE JOB '228d70913eab60dda85c5e7f78b5782c';
++--+--+-+-+
+|

(flink) 01/06: Revert "[FLINK-35189][connectors/filesystem] Modify the visibility of filesystem connector related methods to protected"

2024-06-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 3d829f99e07b6dbaf4b4914c744adc608e30204d
Author: fengli 
AuthorDate: Tue Jun 25 15:17:05 2024 +0800

Revert "[FLINK-35189][connectors/filesystem] Modify the visibility of 
filesystem connector related methods to protected"

This reverts commit e3cda01cac737c97a104feae0e35da1eb91a8751.

(cherry picked from commit e9eabb16f2ce85f2d039b8c0e94d2a47b199e444)
---
 .../apache/flink/connector/file/src/AbstractFileSource.java   | 11 +++
 .../flink/connector/file/table/FileSystemTableFactory.java|  4 ++--
 .../flink/connector/file/table/FileSystemTableSource.java |  4 ++--
 3 files changed, 7 insertions(+), 12 deletions(-)

diff --git 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/AbstractFileSource.java
 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/AbstractFileSource.java
index c9133172070..50dedbdeedf 100644
--- 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/AbstractFileSource.java
+++ 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/AbstractFileSource.java
@@ -147,7 +147,7 @@ public abstract class AbstractFileSource
 throw new FlinkRuntimeException("Could not enumerate file splits", 
e);
 }
 
-return createSplitEnumerator(getBoundedness(), enumContext, 
enumerator, splits, null);
+return createSplitEnumerator(enumContext, enumerator, splits, null);
 }
 
 @Override
@@ -164,11 +164,7 @@ public abstract class AbstractFileSource
 (Collection) checkpoint.getSplits();
 
 return createSplitEnumerator(
-getBoundedness(),
-enumContext,
-enumerator,
-splits,
-checkpoint.getAlreadyProcessedPaths());
+enumContext, enumerator, splits, 
checkpoint.getAlreadyProcessedPaths());
 }
 
 @Override
@@ -190,7 +186,6 @@ public abstract class AbstractFileSource
 // 
 
 private SplitEnumerator> 
createSplitEnumerator(
-Boundedness boundedness,
 SplitEnumeratorContext context,
 FileEnumerator enumerator,
 Collection splits,
@@ -204,7 +199,7 @@ public abstract class AbstractFileSource
 
 final FileSplitAssigner splitAssigner = assignerFactory.create(splits);
 
-if (Boundedness.BOUNDED == boundedness) {
+if (continuousEnumerationSettings == null) {
 // bounded case
 return castGeneric(new StaticFileSplitEnumerator(fileSplitContext, 
splitAssigner));
 } else {
diff --git 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
index 00db848356b..31f6dddbc69 100644
--- 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
+++ 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
@@ -149,7 +149,7 @@ public class FileSystemTableFactory implements 
DynamicTableSourceFactory, Dynami
 .collect(Collectors.toSet());
 }
 
-protected void validate(FactoryUtil.TableFactoryHelper helper) {
+private void validate(FactoryUtil.TableFactoryHelper helper) {
 // Except format options, some formats like parquet and orc can not 
list all supported
 // options.
 helper.validateExcept(helper.getOptions().get(FactoryUtil.FORMAT) + 
".");
@@ -160,7 +160,7 @@ public class FileSystemTableFactory implements 
DynamicTableSourceFactory, Dynami
 
.get(FileSystemConnectorOptions.SINK_PARTITION_COMMIT_WATERMARK_TIME_ZONE));
 }
 
-protected > DecodingFormat 
discoverDecodingFormat(
+private > DecodingFormat 
discoverDecodingFormat(
 Context context, Class formatFactoryClass) {
 FactoryUtil.TableFactoryHelper helper = 
FactoryUtil.createTableFactoryHelper(this, context);
 if (formatFactoryExists(context, formatFactoryClass)) {
diff --git 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java
 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java
index bbaa2a310d5..1e53eb53cfb 100644
--- 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java
+++ 
b/flink-connectors/flink-c

(flink) branch release-1.20 updated (01033503e7c -> 83e6521c3de)

2024-06-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git


from 01033503e7c [FLINK-35692][docs] Fix the default version of sql gateway 
OpenAPI
 new 3d829f99e07 Revert "[FLINK-35189][connectors/filesystem] Modify the 
visibility of filesystem connector related methods to protected"
 new 6ac8697dd5d [FLINK-35691][testutils] Optimize test-filesystem logic to 
support streaming and batch read
 new f2a4c5ac066 [FLINK-35691][table] Fix partition.fields.#.date-formatter 
option verify logic
 new c2c367b761a [FLINK-35691][table] Fix unexpected behavior of repeated 
suspend and resume materialized table
 new 54bd0552025 [FLINK-35691][table] Fix drop table statement can drop 
materialized table
 new 83e6521c3de [FLINK-35691][table] Fix cluster info displayed 
incorrectly on SqlClient when RowFormat is PLAIN_TEXT

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../connector/file/src/AbstractFileSource.java |  11 +-
 .../file/table/FileSystemTableSource.java  |   4 +-
 .../MaterializedTableManager.java  |  63 +-
 .../service/MaterializedTableStatementITCase.java  | 229 -
 .../table/api/internal/StaticResultProvider.java   |  18 ++
 .../apache/flink/table/catalog/CatalogManager.java |  30 ++-
 .../DropMaterializedTableOperation.java|  27 ++-
 .../SqlCreateMaterializedTableConverter.java   |  49 -
 ...erializedTableNodeToOperationConverterTest.java |  68 ++
 .../flink/connector/file/src/TestFileSource.java   | 190 -
 .../file/table/TestFileSystemTableSource.java  |  88 
 .../file/testutils/TestFileSystemTableFactory.java |  13 +-
 .../testutils/TestFileSystemTableFactoryTest.java  |  14 +-
 13 files changed, 487 insertions(+), 317 deletions(-)
 delete mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/src/TestFileSource.java
 delete mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/table/TestFileSystemTableSource.java



(flink) 05/06: [FLINK-35691][table] Fix drop table statement can drop materialized table

2024-06-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 54bd055202527a1c4242422939e1ae86cd0f05bf
Author: fengli 
AuthorDate: Wed Jun 26 13:31:18 2024 +0800

[FLINK-35691][table] Fix drop table statement can drop materialized table

(cherry picked from commit 132d2fe24f18f27d3f752e96ea43ee7c6d196418)
---
 .../service/MaterializedTableStatementITCase.java  | 46 ++
 .../apache/flink/table/catalog/CatalogManager.java | 30 ++
 .../DropMaterializedTableOperation.java| 27 +++--
 3 files changed, 92 insertions(+), 11 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
 
b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
index 98ee00d48ee..6ed91b954af 100644
--- 
a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
+++ 
b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
@@ -991,6 +991,29 @@ public class MaterializedTableStatementITCase extends 
AbstractMaterializedTableS
 List jobResults = fetchAllResults(service, sessionHandle, 
describeJobHandle);
 
assertThat(jobResults.get(0).getString(2).toString()).isEqualTo("RUNNING");
 
+// Drop materialized table using drop table statement
+String dropTableUsingMaterializedTableDDL = "DROP TABLE users_shops";
+OperationHandle dropTableUsingMaterializedTableHandle =
+service.executeStatement(
+sessionHandle, dropTableUsingMaterializedTableDDL, -1, 
new Configuration());
+
+assertThatThrownBy(
+() ->
+awaitOperationTermination(
+service,
+sessionHandle,
+dropTableUsingMaterializedTableHandle))
+.rootCause()
+.isInstanceOf(ValidationException.class)
+.hasMessage(
+String.format(
+"Table with identifier '%s' does not exist.",
+ObjectIdentifier.of(
+fileSystemCatalogName,
+TEST_DEFAULT_DATABASE,
+"users_shops")
+.asSummaryString()));
+
 // drop materialized table
 String dropMaterializedTableDDL = "DROP MATERIALIZED TABLE IF EXISTS 
users_shops";
 OperationHandle dropMaterializedTableHandle =
@@ -1091,6 +1114,29 @@ public class MaterializedTableStatementITCase extends 
AbstractMaterializedTableS
 // verify refresh workflow is created
 
assertThat(embeddedWorkflowScheduler.getQuartzScheduler().checkExists(jobKey)).isTrue();
 
+// Drop materialized table using drop table statement
+String dropTableUsingMaterializedTableDDL = "DROP TABLE users_shops";
+OperationHandle dropTableUsingMaterializedTableHandle =
+service.executeStatement(
+sessionHandle, dropTableUsingMaterializedTableDDL, -1, 
new Configuration());
+
+assertThatThrownBy(
+() ->
+awaitOperationTermination(
+service,
+sessionHandle,
+dropTableUsingMaterializedTableHandle))
+.rootCause()
+.isInstanceOf(ValidationException.class)
+.hasMessage(
+String.format(
+"Table with identifier '%s' does not exist.",
+ObjectIdentifier.of(
+fileSystemCatalogName,
+TEST_DEFAULT_DATABASE,
+"users_shops")
+.asSummaryString()));
+
 // drop materialized table
 String dropMaterializedTableDDL = "DROP MATERIALIZED TABLE IF EXISTS 
users_shops";
 OperationHandle dropMaterializedTableHandle =
diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java
index 5491df3f79e..bba3673c435 100644
--- 
a/

(flink) 03/06: [FLINK-35691][table] Fix partition.fields.#.date-formatter option verify logic

2024-06-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git

commit f2a4c5ac06600bc6f8acf12bc4138513180a685c
Author: fengli 
AuthorDate: Tue Jun 25 19:24:51 2024 +0800

[FLINK-35691][table] Fix partition.fields.#.date-formatter option verify 
logic

(cherry picked from commit 5617d62c98f160af1a4a879cc2d40ef6209b2a32)
---
 .../MaterializedTableManager.java  |  8 +--
 .../service/MaterializedTableStatementITCase.java  |  2 +-
 .../SqlCreateMaterializedTableConverter.java   | 49 +++-
 ...erializedTableNodeToOperationConverterTest.java | 68 ++
 4 files changed, 120 insertions(+), 7 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
index 036d610af81..a4cd306db60 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
@@ -581,7 +581,7 @@ public class MaterializedTableManager {
 
 try {
 LOG.info(
-"Begin to manually refreshing the materialized table {}, 
statement: {}",
+"Begin to refreshing the materialized table {}, statement: 
{}",
 materializedTableIdentifier,
 insertStatement);
 ResultFetcher resultFetcher =
@@ -610,7 +610,7 @@ public class MaterializedTableManager {
 } catch (Exception e) {
 throw new SqlExecutionException(
 String.format(
-"Manually refreshing the materialized table %s 
occur exception.",
+"Refreshing the materialized table %s occur 
exception.",
 materializedTableIdentifier),
 e);
 }
@@ -697,8 +697,8 @@ public class MaterializedTableManager {
 if (!nonStringPartitionKeys.isEmpty()) {
 throw new ValidationException(
 String.format(
-"Currently, manually refreshing materialized table 
only supports specifying char and string type"
-+ " partition keys. All specific partition 
keys with unsupported types are:\n\n%s",
+"Currently, refreshing materialized table only 
supports referring to char, varchar and string type"
++ " partition keys. All specified 
partition keys in partition specs with unsupported types are:\n\n%s",
 String.join("\n", nonStringPartitionKeys)));
 }
 }
diff --git 
a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
 
b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
index 961a949ec83..f090a6cada7 100644
--- 
a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
+++ 
b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
@@ -444,7 +444,7 @@ public class MaterializedTableStatementITCase extends 
AbstractMaterializedTableS
 .rootCause()
 .isInstanceOf(ValidationException.class)
 .hasMessage(
-"Currently, manually refreshing materialized table 
only supports specifying char and string type partition keys. All specific 
partition keys with unsupported types are:\n"
+"Currently, refreshing materialized table only 
supports referring to char, varchar and string type partition keys. All 
specified partition keys in partition specs with unsupported types are:\n"
 + "\n"
 + "ds2");
 }
diff --git 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateMaterializedTableConverter.java
 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateMaterializedTableConverter.java
index b48070d50e7..de14add9692 100644
--- 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateMaterializedTableConverter.java
+++ 
b/flink-table/flink-table-planner/sr

(flink) 06/06: [FLINK-35691][table] Fix cluster info displayed incorrectly on SqlClient when RowFormat is PLAIN_TEXT

2024-06-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 83e6521c3de4b4333df91e8448801918bad901e8
Author: fengli 
AuthorDate: Wed Jun 26 14:29:11 2024 +0800

[FLINK-35691][table] Fix cluster info displayed incorrectly on SqlClient 
when RowFormat is PLAIN_TEXT

(cherry picked from commit 27287a105f6585e89795e2a6cbffa8254abb6e57)
---
 .../flink/table/api/internal/StaticResultProvider.java | 18 ++
 1 file changed, 18 insertions(+)

diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/StaticResultProvider.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/StaticResultProvider.java
index c9eb3f776a8..fda247683c9 100644
--- 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/StaticResultProvider.java
+++ 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/StaticResultProvider.java
@@ -22,6 +22,8 @@ import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.core.execution.JobClient;
 import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
 import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.StringData;
@@ -30,8 +32,10 @@ import 
org.apache.flink.table.utils.print.RowDataToStringConverter;
 import org.apache.flink.types.Row;
 import org.apache.flink.util.CloseableIterator;
 
+import java.util.Arrays;
 import java.util.List;
 import java.util.function.Function;
+import java.util.stream.Collectors;
 
 /** Create result provider from a static set of data using external types. */
 @Internal
@@ -51,6 +55,20 @@ public class StaticResultProvider implements ResultProvider {
 results[i] = "TRUE";
 } else if (Boolean.FALSE.equals(value)) {
 results[i] = "FALSE";
+} else if (value instanceof GenericMapData) {
+// TODO: This is a temporary solution, the long-term 
solution is to use
+// RowDataToStringConverterImpl
+GenericMapData mapData = (GenericMapData) value;
+if (mapData.size() == 0) {
+results[i] = PrintStyle.NULL_VALUE;
+} else {
+Object[] keyArr =
+((GenericArrayData) 
mapData.keyArray()).toObjectArray();
+results[i] =
+Arrays.stream(keyArr)
+.map(key -> key + "=" + 
mapData.get(key))
+.collect(Collectors.joining(", ", 
"{", "}"));
+}
 } else {
 results[i] = value == null ? PrintStyle.NULL_VALUE : 
"" + value;
 }



(flink) 02/06: [FLINK-35691][testutils] Optimize test-filesystem logic to support streaming and batch read

2024-06-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 6ac8697dd5d9bb3a17d27cb22f62267938667bae
Author: fengli 
AuthorDate: Tue Jun 25 15:43:55 2024 +0800

[FLINK-35691][testutils] Optimize test-filesystem logic to support 
streaming and batch read

(cherry picked from commit fc6f288d1bc48c633dbdd0db3a795465b83ce3f3)
---
 .../file/table/FileSystemTableFactory.java |   4 +-
 .../flink/connector/file/src/TestFileSource.java   | 190 -
 .../file/table/TestFileSystemTableSource.java  |  88 --
 .../file/testutils/TestFileSystemTableFactory.java |  13 +-
 .../testutils/TestFileSystemTableFactoryTest.java  |  14 +-
 5 files changed, 18 insertions(+), 291 deletions(-)

diff --git 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
index 31f6dddbc69..00db848356b 100644
--- 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
+++ 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
@@ -149,7 +149,7 @@ public class FileSystemTableFactory implements 
DynamicTableSourceFactory, Dynami
 .collect(Collectors.toSet());
 }
 
-private void validate(FactoryUtil.TableFactoryHelper helper) {
+protected void validate(FactoryUtil.TableFactoryHelper helper) {
 // Except format options, some formats like parquet and orc can not 
list all supported
 // options.
 helper.validateExcept(helper.getOptions().get(FactoryUtil.FORMAT) + 
".");
@@ -160,7 +160,7 @@ public class FileSystemTableFactory implements 
DynamicTableSourceFactory, Dynami
 
.get(FileSystemConnectorOptions.SINK_PARTITION_COMMIT_WATERMARK_TIME_ZONE));
 }
 
-private > DecodingFormat 
discoverDecodingFormat(
+protected > DecodingFormat 
discoverDecodingFormat(
 Context context, Class formatFactoryClass) {
 FactoryUtil.TableFactoryHelper helper = 
FactoryUtil.createTableFactoryHelper(this, context);
 if (formatFactoryExists(context, formatFactoryClass)) {
diff --git 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/src/TestFileSource.java
 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/src/TestFileSource.java
deleted file mode 100644
index e939ad51c36..000
--- 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/src/TestFileSource.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.connector.file.src;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.connector.source.Boundedness;
-import org.apache.flink.api.connector.source.DynamicParallelismInference;
-import org.apache.flink.connector.file.src.assigners.FileSplitAssigner;
-import 
org.apache.flink.connector.file.src.assigners.LocalityAwareSplitAssigner;
-import 
org.apache.flink.connector.file.src.enumerate.BlockSplittingRecursiveEnumerator;
-import org.apache.flink.connector.file.src.enumerate.FileEnumerator;
-import 
org.apache.flink.connector.file.src.enumerate.NonSplittingRecursiveEnumerator;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.io.SimpleVersionedSerializer;
-import org.apache.flink.util.FlinkRuntimeException;
-import org.apache.flink.util.Preconditions;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.util.Collection;
-
-/**
- * A unified data source that reads files - both in batch and in streaming 
mode. This is used only
- * for test. Due to {@link FileSource} is a final class, so we can't extend it 
directl

(flink) 04/06: [FLINK-35691][table] Fix unexpected behavior of repeated suspend and resume materialized table

2024-06-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git

commit c2c367b761a32e925bf6062e47646c828b282f76
Author: fengli 
AuthorDate: Tue Jun 25 20:46:07 2024 +0800

[FLINK-35691][table] Fix unexpected behavior of repeated suspend and resume 
materialized table

(cherry picked from commit 24040de22af994e624b3fa14dac8b68e2c25a4c8)
---
 .../MaterializedTableManager.java  |  55 +++
 .../service/MaterializedTableStatementITCase.java  | 181 +
 2 files changed, 236 insertions(+)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
index a4cd306db60..293634e66d1 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
@@ -295,6 +295,15 @@ public class MaterializedTableManager {
 CatalogMaterializedTable materializedTable =
 getCatalogMaterializedTable(operationExecutor, 
tableIdentifier);
 
+// Initialization phase doesn't support resume operation.
+if (CatalogMaterializedTable.RefreshStatus.INITIALIZING
+== materializedTable.getRefreshStatus()) {
+throw new SqlExecutionException(
+String.format(
+"Materialized table %s is being initialized and 
does not support suspend operation.",
+tableIdentifier));
+}
+
 if (CatalogMaterializedTable.RefreshMode.CONTINUOUS == 
materializedTable.getRefreshMode()) {
 suspendContinuousRefreshJob(
 operationExecutor, handle, tableIdentifier, 
materializedTable);
@@ -313,6 +322,14 @@ public class MaterializedTableManager {
 ContinuousRefreshHandler refreshHandler =
 
deserializeContinuousHandler(materializedTable.getSerializedRefreshHandler());
 
+if (CatalogMaterializedTable.RefreshStatus.SUSPENDED
+== materializedTable.getRefreshStatus()) {
+throw new SqlExecutionException(
+String.format(
+"Materialized table %s continuous refresh job 
has been suspended, jobId is %s.",
+tableIdentifier, refreshHandler.getJobId()));
+}
+
 String savepointPath =
 stopJobWithSavepoint(operationExecutor, handle, 
refreshHandler.getJobId());
 
@@ -344,6 +361,14 @@ public class MaterializedTableManager {
 OperationHandle handle,
 ObjectIdentifier tableIdentifier,
 CatalogMaterializedTable materializedTable) {
+if (CatalogMaterializedTable.RefreshStatus.SUSPENDED
+== materializedTable.getRefreshStatus()) {
+throw new SqlExecutionException(
+String.format(
+"Materialized table %s refresh workflow has been 
suspended.",
+tableIdentifier));
+}
+
 if (workflowScheduler == null) {
 throw new SqlExecutionException(
 "The workflow scheduler must be configured when suspending 
materialized table in full refresh mode.");
@@ -384,6 +409,15 @@ public class MaterializedTableManager {
 CatalogMaterializedTable catalogMaterializedTable =
 getCatalogMaterializedTable(operationExecutor, 
tableIdentifier);
 
+// Initialization phase doesn't support resume operation.
+if (CatalogMaterializedTable.RefreshStatus.INITIALIZING
+== catalogMaterializedTable.getRefreshStatus()) {
+throw new SqlExecutionException(
+String.format(
+"Materialized table %s is being initialized and 
does not support resume operation.",
+tableIdentifier));
+}
+
 if (CatalogMaterializedTable.RefreshMode.CONTINUOUS
 == catalogMaterializedTable.getRefreshMode()) {
 resumeContinuousRefreshJob(
@@ -414,6 +448,18 @@ public class MaterializedTableManager {
 deserializeContinuousHandler(
 
catalogMaterializedTable.getSerializedRefreshHandler());
 
+// Repeated resume continuous refresh job is not supported
+if (CatalogMaterializedTable.RefreshStatus.ACTIVATED
+== catalogMaterializedTable.getRefreshStatus()) {
+JobStatus jobStat

(flink) 02/06: [FLINK-35691][testutils] Optimize test-filesystem logic to support streaming and batch read

2024-06-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit fc6f288d1bc48c633dbdd0db3a795465b83ce3f3
Author: fengli 
AuthorDate: Tue Jun 25 15:43:55 2024 +0800

[FLINK-35691][testutils] Optimize test-filesystem logic to support 
streaming and batch read
---
 .../file/table/FileSystemTableFactory.java |   4 +-
 .../flink/connector/file/src/TestFileSource.java   | 190 -
 .../file/table/TestFileSystemTableSource.java  |  88 --
 .../file/testutils/TestFileSystemTableFactory.java |  13 +-
 .../testutils/TestFileSystemTableFactoryTest.java  |  14 +-
 5 files changed, 18 insertions(+), 291 deletions(-)

diff --git 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
index 31f6dddbc69..00db848356b 100644
--- 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
+++ 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
@@ -149,7 +149,7 @@ public class FileSystemTableFactory implements 
DynamicTableSourceFactory, Dynami
 .collect(Collectors.toSet());
 }
 
-private void validate(FactoryUtil.TableFactoryHelper helper) {
+protected void validate(FactoryUtil.TableFactoryHelper helper) {
 // Except format options, some formats like parquet and orc can not 
list all supported
 // options.
 helper.validateExcept(helper.getOptions().get(FactoryUtil.FORMAT) + 
".");
@@ -160,7 +160,7 @@ public class FileSystemTableFactory implements 
DynamicTableSourceFactory, Dynami
 
.get(FileSystemConnectorOptions.SINK_PARTITION_COMMIT_WATERMARK_TIME_ZONE));
 }
 
-private > DecodingFormat 
discoverDecodingFormat(
+protected > DecodingFormat 
discoverDecodingFormat(
 Context context, Class formatFactoryClass) {
 FactoryUtil.TableFactoryHelper helper = 
FactoryUtil.createTableFactoryHelper(this, context);
 if (formatFactoryExists(context, formatFactoryClass)) {
diff --git 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/src/TestFileSource.java
 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/src/TestFileSource.java
deleted file mode 100644
index e939ad51c36..000
--- 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/src/TestFileSource.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.connector.file.src;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.connector.source.Boundedness;
-import org.apache.flink.api.connector.source.DynamicParallelismInference;
-import org.apache.flink.connector.file.src.assigners.FileSplitAssigner;
-import 
org.apache.flink.connector.file.src.assigners.LocalityAwareSplitAssigner;
-import 
org.apache.flink.connector.file.src.enumerate.BlockSplittingRecursiveEnumerator;
-import org.apache.flink.connector.file.src.enumerate.FileEnumerator;
-import 
org.apache.flink.connector.file.src.enumerate.NonSplittingRecursiveEnumerator;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.io.SimpleVersionedSerializer;
-import org.apache.flink.util.FlinkRuntimeException;
-import org.apache.flink.util.Preconditions;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.util.Collection;
-
-/**
- * A unified data source that reads files - both in batch and in streaming 
mode. This is used only
- * for test. Due to {@link FileSource} is a final class, so we can't extend it 
directly.
- *
- * @param  The type of the events/records produced by this source.
- */
-@Inte

(flink) 04/06: [FLINK-35691][table] Fix unexpected behavior of repeated suspend and resume materialized table

2024-06-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 24040de22af994e624b3fa14dac8b68e2c25a4c8
Author: fengli 
AuthorDate: Tue Jun 25 20:46:07 2024 +0800

[FLINK-35691][table] Fix unexpected behavior of repeated suspend and resume 
materialized table
---
 .../MaterializedTableManager.java  |  55 +++
 .../service/MaterializedTableStatementITCase.java  | 181 +
 2 files changed, 236 insertions(+)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
index a4cd306db60..293634e66d1 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
@@ -295,6 +295,15 @@ public class MaterializedTableManager {
 CatalogMaterializedTable materializedTable =
 getCatalogMaterializedTable(operationExecutor, 
tableIdentifier);
 
+// Initialization phase doesn't support resume operation.
+if (CatalogMaterializedTable.RefreshStatus.INITIALIZING
+== materializedTable.getRefreshStatus()) {
+throw new SqlExecutionException(
+String.format(
+"Materialized table %s is being initialized and 
does not support suspend operation.",
+tableIdentifier));
+}
+
 if (CatalogMaterializedTable.RefreshMode.CONTINUOUS == 
materializedTable.getRefreshMode()) {
 suspendContinuousRefreshJob(
 operationExecutor, handle, tableIdentifier, 
materializedTable);
@@ -313,6 +322,14 @@ public class MaterializedTableManager {
 ContinuousRefreshHandler refreshHandler =
 
deserializeContinuousHandler(materializedTable.getSerializedRefreshHandler());
 
+if (CatalogMaterializedTable.RefreshStatus.SUSPENDED
+== materializedTable.getRefreshStatus()) {
+throw new SqlExecutionException(
+String.format(
+"Materialized table %s continuous refresh job 
has been suspended, jobId is %s.",
+tableIdentifier, refreshHandler.getJobId()));
+}
+
 String savepointPath =
 stopJobWithSavepoint(operationExecutor, handle, 
refreshHandler.getJobId());
 
@@ -344,6 +361,14 @@ public class MaterializedTableManager {
 OperationHandle handle,
 ObjectIdentifier tableIdentifier,
 CatalogMaterializedTable materializedTable) {
+if (CatalogMaterializedTable.RefreshStatus.SUSPENDED
+== materializedTable.getRefreshStatus()) {
+throw new SqlExecutionException(
+String.format(
+"Materialized table %s refresh workflow has been 
suspended.",
+tableIdentifier));
+}
+
 if (workflowScheduler == null) {
 throw new SqlExecutionException(
 "The workflow scheduler must be configured when suspending 
materialized table in full refresh mode.");
@@ -384,6 +409,15 @@ public class MaterializedTableManager {
 CatalogMaterializedTable catalogMaterializedTable =
 getCatalogMaterializedTable(operationExecutor, 
tableIdentifier);
 
+// Initialization phase doesn't support resume operation.
+if (CatalogMaterializedTable.RefreshStatus.INITIALIZING
+== catalogMaterializedTable.getRefreshStatus()) {
+throw new SqlExecutionException(
+String.format(
+"Materialized table %s is being initialized and 
does not support resume operation.",
+tableIdentifier));
+}
+
 if (CatalogMaterializedTable.RefreshMode.CONTINUOUS
 == catalogMaterializedTable.getRefreshMode()) {
 resumeContinuousRefreshJob(
@@ -414,6 +448,18 @@ public class MaterializedTableManager {
 deserializeContinuousHandler(
 
catalogMaterializedTable.getSerializedRefreshHandler());
 
+// Repeated resume continuous refresh job is not supported
+if (CatalogMaterializedTable.RefreshStatus.ACTIVATED
+== catalogMaterializedTable.getRefreshStatus()) {
+JobStatus jobStatus = getJobStatus(operationExecutor, handle, 
refreshHandler);
+if (!jobSt

(flink) 03/06: [FLINK-35691][table] Fix partition.fields.#.date-formatter option verify logic

2024-06-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 5617d62c98f160af1a4a879cc2d40ef6209b2a32
Author: fengli 
AuthorDate: Tue Jun 25 19:24:51 2024 +0800

[FLINK-35691][table] Fix partition.fields.#.date-formatter option verify 
logic
---
 .../MaterializedTableManager.java  |  8 +--
 .../service/MaterializedTableStatementITCase.java  |  2 +-
 .../SqlCreateMaterializedTableConverter.java   | 49 +++-
 ...erializedTableNodeToOperationConverterTest.java | 68 ++
 4 files changed, 120 insertions(+), 7 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
index 036d610af81..a4cd306db60 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
@@ -581,7 +581,7 @@ public class MaterializedTableManager {
 
 try {
 LOG.info(
-"Begin to manually refreshing the materialized table {}, 
statement: {}",
+"Begin to refreshing the materialized table {}, statement: 
{}",
 materializedTableIdentifier,
 insertStatement);
 ResultFetcher resultFetcher =
@@ -610,7 +610,7 @@ public class MaterializedTableManager {
 } catch (Exception e) {
 throw new SqlExecutionException(
 String.format(
-"Manually refreshing the materialized table %s 
occur exception.",
+"Refreshing the materialized table %s occur 
exception.",
 materializedTableIdentifier),
 e);
 }
@@ -697,8 +697,8 @@ public class MaterializedTableManager {
 if (!nonStringPartitionKeys.isEmpty()) {
 throw new ValidationException(
 String.format(
-"Currently, manually refreshing materialized table 
only supports specifying char and string type"
-+ " partition keys. All specific partition 
keys with unsupported types are:\n\n%s",
+"Currently, refreshing materialized table only 
supports referring to char, varchar and string type"
++ " partition keys. All specified 
partition keys in partition specs with unsupported types are:\n\n%s",
 String.join("\n", nonStringPartitionKeys)));
 }
 }
diff --git 
a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
 
b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
index 961a949ec83..f090a6cada7 100644
--- 
a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
+++ 
b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
@@ -444,7 +444,7 @@ public class MaterializedTableStatementITCase extends 
AbstractMaterializedTableS
 .rootCause()
 .isInstanceOf(ValidationException.class)
 .hasMessage(
-"Currently, manually refreshing materialized table 
only supports specifying char and string type partition keys. All specific 
partition keys with unsupported types are:\n"
+"Currently, refreshing materialized table only 
supports referring to char, varchar and string type partition keys. All 
specified partition keys in partition specs with unsupported types are:\n"
 + "\n"
 + "ds2");
 }
diff --git 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateMaterializedTableConverter.java
 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateMaterializedTableConverter.java
index b48070d50e7..de14add9692 100644
--- 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateMaterializedTableConverter.java
+++ 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateMateriali

(flink) 06/06: [FLINK-35691][table] Fix cluster info displayed incorrectly on SqlClient when RowFormat is PLAIN_TEXT

2024-06-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 27287a105f6585e89795e2a6cbffa8254abb6e57
Author: fengli 
AuthorDate: Wed Jun 26 14:29:11 2024 +0800

[FLINK-35691][table] Fix cluster info displayed incorrectly on SqlClient 
when RowFormat is PLAIN_TEXT
---
 .../flink/table/api/internal/StaticResultProvider.java | 18 ++
 1 file changed, 18 insertions(+)

diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/StaticResultProvider.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/StaticResultProvider.java
index c9eb3f776a8..fda247683c9 100644
--- 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/StaticResultProvider.java
+++ 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/StaticResultProvider.java
@@ -22,6 +22,8 @@ import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.core.execution.JobClient;
 import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
 import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.StringData;
@@ -30,8 +32,10 @@ import 
org.apache.flink.table.utils.print.RowDataToStringConverter;
 import org.apache.flink.types.Row;
 import org.apache.flink.util.CloseableIterator;
 
+import java.util.Arrays;
 import java.util.List;
 import java.util.function.Function;
+import java.util.stream.Collectors;
 
 /** Create result provider from a static set of data using external types. */
 @Internal
@@ -51,6 +55,20 @@ public class StaticResultProvider implements ResultProvider {
 results[i] = "TRUE";
 } else if (Boolean.FALSE.equals(value)) {
 results[i] = "FALSE";
+} else if (value instanceof GenericMapData) {
+// TODO: This is a temporary solution, the long-term 
solution is to use
+// RowDataToStringConverterImpl
+GenericMapData mapData = (GenericMapData) value;
+if (mapData.size() == 0) {
+results[i] = PrintStyle.NULL_VALUE;
+} else {
+Object[] keyArr =
+((GenericArrayData) 
mapData.keyArray()).toObjectArray();
+results[i] =
+Arrays.stream(keyArr)
+.map(key -> key + "=" + 
mapData.get(key))
+.collect(Collectors.joining(", ", 
"{", "}"));
+}
 } else {
 results[i] = value == null ? PrintStyle.NULL_VALUE : 
"" + value;
 }



(flink) 05/06: [FLINK-35691][table] Fix drop table statement can drop materialized table

2024-06-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 132d2fe24f18f27d3f752e96ea43ee7c6d196418
Author: fengli 
AuthorDate: Wed Jun 26 13:31:18 2024 +0800

[FLINK-35691][table] Fix drop table statement can drop materialized table
---
 .../service/MaterializedTableStatementITCase.java  | 46 ++
 .../apache/flink/table/catalog/CatalogManager.java | 30 ++
 .../DropMaterializedTableOperation.java| 27 +++--
 3 files changed, 92 insertions(+), 11 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
 
b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
index 98ee00d48ee..6ed91b954af 100644
--- 
a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
+++ 
b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
@@ -991,6 +991,29 @@ public class MaterializedTableStatementITCase extends 
AbstractMaterializedTableS
 List jobResults = fetchAllResults(service, sessionHandle, 
describeJobHandle);
 
assertThat(jobResults.get(0).getString(2).toString()).isEqualTo("RUNNING");
 
+// Drop materialized table using drop table statement
+String dropTableUsingMaterializedTableDDL = "DROP TABLE users_shops";
+OperationHandle dropTableUsingMaterializedTableHandle =
+service.executeStatement(
+sessionHandle, dropTableUsingMaterializedTableDDL, -1, 
new Configuration());
+
+assertThatThrownBy(
+() ->
+awaitOperationTermination(
+service,
+sessionHandle,
+dropTableUsingMaterializedTableHandle))
+.rootCause()
+.isInstanceOf(ValidationException.class)
+.hasMessage(
+String.format(
+"Table with identifier '%s' does not exist.",
+ObjectIdentifier.of(
+fileSystemCatalogName,
+TEST_DEFAULT_DATABASE,
+"users_shops")
+.asSummaryString()));
+
 // drop materialized table
 String dropMaterializedTableDDL = "DROP MATERIALIZED TABLE IF EXISTS 
users_shops";
 OperationHandle dropMaterializedTableHandle =
@@ -1091,6 +1114,29 @@ public class MaterializedTableStatementITCase extends 
AbstractMaterializedTableS
 // verify refresh workflow is created
 
assertThat(embeddedWorkflowScheduler.getQuartzScheduler().checkExists(jobKey)).isTrue();
 
+// Drop materialized table using drop table statement
+String dropTableUsingMaterializedTableDDL = "DROP TABLE users_shops";
+OperationHandle dropTableUsingMaterializedTableHandle =
+service.executeStatement(
+sessionHandle, dropTableUsingMaterializedTableDDL, -1, 
new Configuration());
+
+assertThatThrownBy(
+() ->
+awaitOperationTermination(
+service,
+sessionHandle,
+dropTableUsingMaterializedTableHandle))
+.rootCause()
+.isInstanceOf(ValidationException.class)
+.hasMessage(
+String.format(
+"Table with identifier '%s' does not exist.",
+ObjectIdentifier.of(
+fileSystemCatalogName,
+TEST_DEFAULT_DATABASE,
+"users_shops")
+.asSummaryString()));
+
 // drop materialized table
 String dropMaterializedTableDDL = "DROP MATERIALIZED TABLE IF EXISTS 
users_shops";
 OperationHandle dropMaterializedTableHandle =
diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java
index 5491df3f79e..bba3673c435 100644
--- 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/ta

(flink) 01/06: Revert "[FLINK-35189][connectors/filesystem] Modify the visibility of filesystem connector related methods to protected"

2024-06-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit e9eabb16f2ce85f2d039b8c0e94d2a47b199e444
Author: fengli 
AuthorDate: Tue Jun 25 15:17:05 2024 +0800

Revert "[FLINK-35189][connectors/filesystem] Modify the visibility of 
filesystem connector related methods to protected"

This reverts commit e3cda01cac737c97a104feae0e35da1eb91a8751.
---
 .../apache/flink/connector/file/src/AbstractFileSource.java   | 11 +++
 .../flink/connector/file/table/FileSystemTableFactory.java|  4 ++--
 .../flink/connector/file/table/FileSystemTableSource.java |  4 ++--
 3 files changed, 7 insertions(+), 12 deletions(-)

diff --git 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/AbstractFileSource.java
 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/AbstractFileSource.java
index c9133172070..50dedbdeedf 100644
--- 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/AbstractFileSource.java
+++ 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/AbstractFileSource.java
@@ -147,7 +147,7 @@ public abstract class AbstractFileSource
 throw new FlinkRuntimeException("Could not enumerate file splits", 
e);
 }
 
-return createSplitEnumerator(getBoundedness(), enumContext, 
enumerator, splits, null);
+return createSplitEnumerator(enumContext, enumerator, splits, null);
 }
 
 @Override
@@ -164,11 +164,7 @@ public abstract class AbstractFileSource
 (Collection) checkpoint.getSplits();
 
 return createSplitEnumerator(
-getBoundedness(),
-enumContext,
-enumerator,
-splits,
-checkpoint.getAlreadyProcessedPaths());
+enumContext, enumerator, splits, 
checkpoint.getAlreadyProcessedPaths());
 }
 
 @Override
@@ -190,7 +186,6 @@ public abstract class AbstractFileSource
 // 
 
 private SplitEnumerator> 
createSplitEnumerator(
-Boundedness boundedness,
 SplitEnumeratorContext context,
 FileEnumerator enumerator,
 Collection splits,
@@ -204,7 +199,7 @@ public abstract class AbstractFileSource
 
 final FileSplitAssigner splitAssigner = assignerFactory.create(splits);
 
-if (Boundedness.BOUNDED == boundedness) {
+if (continuousEnumerationSettings == null) {
 // bounded case
 return castGeneric(new StaticFileSplitEnumerator(fileSplitContext, 
splitAssigner));
 } else {
diff --git 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
index 00db848356b..31f6dddbc69 100644
--- 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
+++ 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
@@ -149,7 +149,7 @@ public class FileSystemTableFactory implements 
DynamicTableSourceFactory, Dynami
 .collect(Collectors.toSet());
 }
 
-protected void validate(FactoryUtil.TableFactoryHelper helper) {
+private void validate(FactoryUtil.TableFactoryHelper helper) {
 // Except format options, some formats like parquet and orc can not 
list all supported
 // options.
 helper.validateExcept(helper.getOptions().get(FactoryUtil.FORMAT) + 
".");
@@ -160,7 +160,7 @@ public class FileSystemTableFactory implements 
DynamicTableSourceFactory, Dynami
 
.get(FileSystemConnectorOptions.SINK_PARTITION_COMMIT_WATERMARK_TIME_ZONE));
 }
 
-protected > DecodingFormat 
discoverDecodingFormat(
+private > DecodingFormat 
discoverDecodingFormat(
 Context context, Class formatFactoryClass) {
 FactoryUtil.TableFactoryHelper helper = 
FactoryUtil.createTableFactoryHelper(this, context);
 if (formatFactoryExists(context, formatFactoryClass)) {
diff --git 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java
 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java
index bbaa2a310d5..1e53eb53cfb 100644
--- 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java
+++ 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTabl

(flink) branch master updated (90508a00bb3 -> 27287a105f6)

2024-06-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 90508a00bb3 [FLINK-35687] JSON_QUERY should return a well formatted 
nested objects/arrays for ARRAY (#24976)
 new e9eabb16f2c Revert "[FLINK-35189][connectors/filesystem] Modify the 
visibility of filesystem connector related methods to protected"
 new fc6f288d1bc [FLINK-35691][testutils] Optimize test-filesystem logic to 
support streaming and batch read
 new 5617d62c98f [FLINK-35691][table] Fix partition.fields.#.date-formatter 
option verify logic
 new 24040de22af [FLINK-35691][table] Fix unexpected behavior of repeated 
suspend and resume materialized table
 new 132d2fe24f1 [FLINK-35691][table] Fix drop table statement can drop 
materialized table
 new 27287a105f6 [FLINK-35691][table] Fix cluster info displayed 
incorrectly on SqlClient when RowFormat is PLAIN_TEXT

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../connector/file/src/AbstractFileSource.java |  11 +-
 .../file/table/FileSystemTableSource.java  |   4 +-
 .../MaterializedTableManager.java  |  63 +-
 .../service/MaterializedTableStatementITCase.java  | 229 -
 .../table/api/internal/StaticResultProvider.java   |  18 ++
 .../apache/flink/table/catalog/CatalogManager.java |  30 ++-
 .../DropMaterializedTableOperation.java|  27 ++-
 .../SqlCreateMaterializedTableConverter.java   |  49 -
 ...erializedTableNodeToOperationConverterTest.java |  68 ++
 .../flink/connector/file/src/TestFileSource.java   | 190 -
 .../file/table/TestFileSystemTableSource.java  |  88 
 .../file/testutils/TestFileSystemTableFactory.java |  13 +-
 .../testutils/TestFileSystemTableFactoryTest.java  |  14 +-
 13 files changed, 487 insertions(+), 317 deletions(-)
 delete mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/src/TestFileSource.java
 delete mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/table/TestFileSystemTableSource.java



(flink) branch release-1.20 updated (d93f7421e0d -> 01033503e7c)

2024-06-25 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git


from d93f7421e0d [FLINK-35656][hive] Fix the issue that Hive Source 
incorrectly set max parallelism in dynamic inference mode
 new 9181aaa68d2 [FLINK-35692][docs] Remove 
hugo_extended_0.110.0_Linux-64bit.tar.gz file introduced by wrong operation
 new 01033503e7c [FLINK-35692][docs] Fix the default version of sql gateway 
OpenAPI

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 docs/content.zh/docs/dev/table/sql-gateway/rest.md |   2 +-
 docs/content/docs/dev/table/sql-gateway/rest.md|   2 +-
 docs/hugo_extended_0.110.0_Linux-64bit.tar.gz  | Bin 601126 -> 0 bytes
 3 files changed, 2 insertions(+), 2 deletions(-)
 delete mode 100644 docs/hugo_extended_0.110.0_Linux-64bit.tar.gz



(flink) 01/02: [FLINK-35692][docs] Remove hugo_extended_0.110.0_Linux-64bit.tar.gz file introduced by wrong operation

2024-06-25 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 9181aaa68d2ccb0ef82c573b829c5ce1b9043438
Author: fengli 
AuthorDate: Tue Jun 25 14:44:49 2024 +0800

[FLINK-35692][docs] Remove hugo_extended_0.110.0_Linux-64bit.tar.gz file 
introduced by wrong operation

(cherry picked from commit a6b9881be787394912f68009a53a71f5d3dda6e3)
---
 docs/hugo_extended_0.110.0_Linux-64bit.tar.gz | Bin 601126 -> 0 bytes
 1 file changed, 0 insertions(+), 0 deletions(-)

diff --git a/docs/hugo_extended_0.110.0_Linux-64bit.tar.gz 
b/docs/hugo_extended_0.110.0_Linux-64bit.tar.gz
deleted file mode 100644
index 61791d22a92..000
Binary files a/docs/hugo_extended_0.110.0_Linux-64bit.tar.gz and /dev/null 
differ



(flink) 02/02: [FLINK-35692][docs] Fix the default version of sql gateway OpenAPI

2024-06-25 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.20
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 01033503e7c5d474d71490e5d9f7847405e0c5b0
Author: fengli 
AuthorDate: Tue Jun 25 14:49:10 2024 +0800

[FLINK-35692][docs] Fix the default version of sql gateway OpenAPI

(cherry picked from commit d0ba00072065eb5bdb7d610aef254a69374d8967)
---
 docs/content.zh/docs/dev/table/sql-gateway/rest.md | 2 +-
 docs/content/docs/dev/table/sql-gateway/rest.md| 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/content.zh/docs/dev/table/sql-gateway/rest.md 
b/docs/content.zh/docs/dev/table/sql-gateway/rest.md
index a390f242d2b..6f0cc482021 100644
--- a/docs/content.zh/docs/dev/table/sql-gateway/rest.md
+++ b/docs/content.zh/docs/dev/table/sql-gateway/rest.md
@@ -91,7 +91,7 @@ Endpoint Options
 REST API
 
 
-OpenAPI 规范如下,默认版本是 v2。
+OpenAPI 规范如下,默认版本是 v3。
 
 | Version   | Description|
 | --- ||
diff --git a/docs/content/docs/dev/table/sql-gateway/rest.md 
b/docs/content/docs/dev/table/sql-gateway/rest.md
index 4a4787f030c..eafefaad72b 100644
--- a/docs/content/docs/dev/table/sql-gateway/rest.md
+++ b/docs/content/docs/dev/table/sql-gateway/rest.md
@@ -94,7 +94,7 @@ Endpoint Options
 REST API
 
 
-The available OpenAPI specification is as follows. The default version is v2.
+The available OpenAPI specification is as follows. The default version is v3.
 
 | Version   | Description |
 | --- | --- |



(flink) 01/02: [FLINK-35692][docs] Remove hugo_extended_0.110.0_Linux-64bit.tar.gz file introduced by wrong operation

2024-06-25 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit a6b9881be787394912f68009a53a71f5d3dda6e3
Author: fengli 
AuthorDate: Tue Jun 25 14:44:49 2024 +0800

[FLINK-35692][docs] Remove hugo_extended_0.110.0_Linux-64bit.tar.gz file 
introduced by wrong operation
---
 docs/hugo_extended_0.110.0_Linux-64bit.tar.gz | Bin 601126 -> 0 bytes
 1 file changed, 0 insertions(+), 0 deletions(-)

diff --git a/docs/hugo_extended_0.110.0_Linux-64bit.tar.gz 
b/docs/hugo_extended_0.110.0_Linux-64bit.tar.gz
deleted file mode 100644
index 61791d22a92..000
Binary files a/docs/hugo_extended_0.110.0_Linux-64bit.tar.gz and /dev/null 
differ



(flink) 02/02: [FLINK-35692][docs] Fix the default version of sql gateway OpenAPI

2024-06-25 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit d0ba00072065eb5bdb7d610aef254a69374d8967
Author: fengli 
AuthorDate: Tue Jun 25 14:49:10 2024 +0800

[FLINK-35692][docs] Fix the default version of sql gateway OpenAPI
---
 docs/content.zh/docs/dev/table/sql-gateway/rest.md | 2 +-
 docs/content/docs/dev/table/sql-gateway/rest.md| 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/content.zh/docs/dev/table/sql-gateway/rest.md 
b/docs/content.zh/docs/dev/table/sql-gateway/rest.md
index a390f242d2b..6f0cc482021 100644
--- a/docs/content.zh/docs/dev/table/sql-gateway/rest.md
+++ b/docs/content.zh/docs/dev/table/sql-gateway/rest.md
@@ -91,7 +91,7 @@ Endpoint Options
 REST API
 
 
-OpenAPI 规范如下,默认版本是 v2。
+OpenAPI 规范如下,默认版本是 v3。
 
 | Version   | Description|
 | --- ||
diff --git a/docs/content/docs/dev/table/sql-gateway/rest.md 
b/docs/content/docs/dev/table/sql-gateway/rest.md
index 4a4787f030c..eafefaad72b 100644
--- a/docs/content/docs/dev/table/sql-gateway/rest.md
+++ b/docs/content/docs/dev/table/sql-gateway/rest.md
@@ -94,7 +94,7 @@ Endpoint Options
 REST API
 
 
-The available OpenAPI specification is as follows. The default version is v2.
+The available OpenAPI specification is as follows. The default version is v3.
 
 | Version   | Description |
 | --- | --- |



(flink) branch master updated (01c3fd67ac4 -> d0ba0007206)

2024-06-25 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 01c3fd67ac4 [FLINK-35656][hive] Fix the issue that Hive Source 
incorrectly set max parallelism in dynamic inference mode
 new a6b9881be78 [FLINK-35692][docs] Remove 
hugo_extended_0.110.0_Linux-64bit.tar.gz file introduced by wrong operation
 new d0ba0007206 [FLINK-35692][docs] Fix the default version of sql gateway 
OpenAPI

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 docs/content.zh/docs/dev/table/sql-gateway/rest.md |   2 +-
 docs/content/docs/dev/table/sql-gateway/rest.md|   2 +-
 docs/hugo_extended_0.110.0_Linux-64bit.tar.gz  | Bin 601126 -> 0 bytes
 3 files changed, 2 insertions(+), 2 deletions(-)
 delete mode 100644 docs/hugo_extended_0.110.0_Linux-64bit.tar.gz



(flink) branch master updated (12f2222bbcd -> 662cfd2804c)

2024-06-23 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 12fbbcd [FLINK-35646][docs] Add docs for materialized table 
refresh api
 new 7856190232c [hotfix][table] Fix warning message format in 
WorkflowSchedulerFactoryUtil
 new 662cfd2804c [FLINK-35479][e2e] Add end-to-end test for materialized 
table

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../flink-end-to-end-tests-common/pom.xml  |   8 +-
 .../flink/tests/util/flink/FlinkDistribution.java  |  74 -
 .../flink-sql-gateway-test/pom.xml |   8 +
 .../flink/table/gateway/SqlGatewayE2ECase.java | 309 +
 .../factories/WorkflowSchedulerFactoryUtil.java|   3 +-
 5 files changed, 395 insertions(+), 7 deletions(-)



(flink) 01/02: [hotfix][table] Fix warning message format in WorkflowSchedulerFactoryUtil

2024-06-23 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 7856190232c9678a26ed301bb960652a5015f30d
Author: Feng Jin 
AuthorDate: Thu Jun 20 11:33:07 2024 +0800

[hotfix][table] Fix warning message format in WorkflowSchedulerFactoryUtil
---
 .../org/apache/flink/table/factories/WorkflowSchedulerFactoryUtil.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/WorkflowSchedulerFactoryUtil.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/WorkflowSchedulerFactoryUtil.java
index 593d6b47d6a..e8d1a6333f6 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/WorkflowSchedulerFactoryUtil.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/WorkflowSchedulerFactoryUtil.java
@@ -62,7 +62,8 @@ public class WorkflowSchedulerFactoryUtil {
 String identifier = configuration.get(WORKFLOW_SCHEDULER_TYPE);
 if (StringUtils.isNullOrWhitespaceOnly(identifier)) {
 LOG.warn(
-"Workflow scheduler options do not contain an option key 
'%s' for discovering an workflow scheduler.");
+"Workflow scheduler options do not contain an option key 
'{}' for discovering an workflow scheduler.",
+WORKFLOW_SCHEDULER_TYPE.key());
 return null;
 }
 



(flink) 02/02: [FLINK-35479][e2e] Add end-to-end test for materialized table

2024-06-23 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 662cfd2804c312e667601f877ee00fe547d160e0
Author: Feng Jin 
AuthorDate: Fri Jun 21 14:17:14 2024 +0800

[FLINK-35479][e2e] Add end-to-end test for materialized table
---
 .../flink-end-to-end-tests-common/pom.xml  |   8 +-
 .../flink/tests/util/flink/FlinkDistribution.java  |  74 -
 .../flink-sql-gateway-test/pom.xml |   8 +
 .../flink/table/gateway/SqlGatewayE2ECase.java | 309 +
 4 files changed, 393 insertions(+), 6 deletions(-)

diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml 
b/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml
index 8219669a880..35f7e14799c 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml
@@ -127,7 +127,13 @@ under the License.



-   
+
+org.apache.flink
+flink-sql-gateway
+${project.version}
+provided
+
+
 


diff --git 
a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java
 
b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java
index 92c9986e394..404b1ed4555 100644
--- 
a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java
+++ 
b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java
@@ -23,6 +23,9 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.ConfigurationUtils;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.table.api.ResultKind;
+import org.apache.flink.table.data.RowData;
+import 
org.apache.flink.table.gateway.rest.message.statement.FetchResultsResponseBody;
 import org.apache.flink.test.util.FileUtils;
 import org.apache.flink.test.util.JobSubmission;
 import org.apache.flink.test.util.SQLJobClientMode;
@@ -65,8 +68,11 @@ import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
@@ -463,7 +469,7 @@ public final class FlinkDistribution {
 }
 
 /** This rest client is used to submit SQL strings to Rest Endpoint of Sql 
Gateway. */
-private static class TestSqlGatewayRestClient {
+public static class TestSqlGatewayRestClient {
 
 private final String host;
 private final int port;
@@ -475,12 +481,31 @@ public final class FlinkDistribution {
 this.host = host;
 this.port = port;
 this.version = version;
-this.sessionHandle = openSession();
+this.sessionHandle = openSession(Collections.emptyMap());
 }
 
-private String openSession() throws Exception {
-FormBody.Builder builder = new FormBody.Builder();
-FormBody requestBody = builder.build();
+public TestSqlGatewayRestClient(
+String host, int port, String version, Map 
properties)
+throws Exception {
+this.host = host;
+this.port = port;
+this.version = version;
+this.sessionHandle = openSession(properties);
+}
+
+private String openSession(Map properties) throws 
Exception {
+RequestBody requestBody;
+if (properties == null || properties.isEmpty()) {
+requestBody = new FormBody.Builder().build();
+} else {
+Map requestMap = new HashMap<>();
+requestMap.put("properties", properties);
+requestBody =
+RequestBody.create(
+MediaType.parse("application/json; 
charset=utf-8"),
+OBJECT_MAPPER.writeValueAsString(requestMap));
+}
+
 final Request request =
 new Request.Builder()
 .post(requestBody)
@@ -529,6 +554,45 @@ public final class FlinkDistribution {
 } while (!Objects.equals(status, "FINISHED") && 
!Objects.equals(status, "ERROR"));
 }
 
+public List getOperationResult(String operationHandle) throws 
Exception {
+List res

(flink) 02/03: [FLINK-35646][table] Exclude embedded workflow scheduler related rest api from docs

2024-06-21 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 993c5c12df1c6c3c0b8a51e24a96f0725334fa2e
Author: fengli 
AuthorDate: Fri Jun 21 16:38:45 2024 +0800

[FLINK-35646][table] Exclude embedded workflow scheduler related rest api 
from docs
---
 .../scheduler/CreateEmbeddedSchedulerWorkflowHeaders.java  | 7 +++
 .../scheduler/DeleteEmbeddedSchedulerWorkflowHeaders.java  | 7 +++
 .../scheduler/ResumeEmbeddedSchedulerWorkflowHeaders.java  | 7 +++
 .../scheduler/SuspendEmbeddedSchedulerWorkflowHeaders.java | 7 +++
 4 files changed, 28 insertions(+)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowHeaders.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowHeaders.java
index aa3c20c9559..6dd416f1a9c 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowHeaders.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowHeaders.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.gateway.rest.header.materializedtable.scheduler;
 
+import org.apache.flink.annotation.docs.Documentation;
 import org.apache.flink.runtime.rest.HttpMethodWrapper;
 import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
 import org.apache.flink.runtime.rest.versioning.RestAPIVersion;
@@ -33,6 +34,7 @@ import java.util.Collections;
 import static 
org.apache.flink.table.gateway.rest.util.SqlGatewayRestAPIVersion.V3;
 
 /** Message headers for create workflow in embedded scheduler. */
+@Documentation.ExcludeFromDocumentation("The embedded rest api.")
 public class CreateEmbeddedSchedulerWorkflowHeaders
 implements SqlGatewayMessageHeaders<
 CreateEmbeddedSchedulerWorkflowRequestBody,
@@ -87,4 +89,9 @@ public class CreateEmbeddedSchedulerWorkflowHeaders
 public static CreateEmbeddedSchedulerWorkflowHeaders getInstance() {
 return INSTANCE;
 }
+
+@Override
+public String operationId() {
+return "createWorkflow";
+}
 }
diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/DeleteEmbeddedSchedulerWorkflowHeaders.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/DeleteEmbeddedSchedulerWorkflowHeaders.java
index 5b4e997db15..01677b1a2cc 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/DeleteEmbeddedSchedulerWorkflowHeaders.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/DeleteEmbeddedSchedulerWorkflowHeaders.java
@@ -18,9 +18,11 @@
 
 package org.apache.flink.table.gateway.rest.header.materializedtable.scheduler;
 
+import org.apache.flink.annotation.docs.Documentation;
 import org.apache.flink.runtime.rest.HttpMethodWrapper;
 
 /** Message headers for delete workflow in embedded scheduler. */
+@Documentation.ExcludeFromDocumentation("The embedded rest api.")
 public class DeleteEmbeddedSchedulerWorkflowHeaders
 extends AbstractEmbeddedSchedulerWorkflowHeaders {
 
@@ -47,4 +49,9 @@ public class DeleteEmbeddedSchedulerWorkflowHeaders
 public static DeleteEmbeddedSchedulerWorkflowHeaders getInstance() {
 return INSTANCE;
 }
+
+@Override
+public String operationId() {
+return "deleteWorkflow";
+}
 }
diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/ResumeEmbeddedSchedulerWorkflowHeaders.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/ResumeEmbeddedSchedulerWorkflowHeaders.java
index cc5ad891cb4..59312f58d2a 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/ResumeEmbeddedSchedulerWorkflowHeaders.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/ResumeEmbeddedSchedulerWorkflowHeaders.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.gateway.rest.header.materializedtable.scheduler;
 
+import org.apache.flink.annotation.docs.Documentation;
 import org.apache.flink.runtime.rest.HttpMethodWrapper;
 import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
 import org.apache.fl

(flink) 03/03: [FLINK-35646][docs] Add docs for materialized table refresh api

2024-06-21 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 12fbbcd7919e2378b1a6a9bdf56c2835a79e
Author: fengli 
AuthorDate: Fri Jun 21 16:39:17 2024 +0800

[FLINK-35646][docs] Add docs for materialized table refresh api
---
 docs/content.zh/docs/dev/table/sql-gateway/rest.md |   8 +-
 docs/content/docs/dev/table/sql-gateway/rest.md|   6 +
 docs/hugo_extended_0.110.0_Linux-64bit.tar.gz  | Bin 0 -> 601126 bytes
 .../shortcodes/generated/rest_v3_sql_gateway.html  | 762 +
 docs/static/generated/rest_v3_sql_gateway.yml  | 663 ++
 .../RefreshMaterializedTableHeaders.java   |   5 +
 .../MaterializedTableIdentifierPathParameter.java  |   2 +-
 7 files changed, 1444 insertions(+), 2 deletions(-)

diff --git a/docs/content.zh/docs/dev/table/sql-gateway/rest.md 
b/docs/content.zh/docs/dev/table/sql-gateway/rest.md
index 5caea340bd4..a390f242d2b 100644
--- a/docs/content.zh/docs/dev/table/sql-gateway/rest.md
+++ b/docs/content.zh/docs/dev/table/sql-gateway/rest.md
@@ -96,7 +96,8 @@ OpenAPI 规范如下,默认版本是 v2。
 | Version   | Description|
 | --- ||
 | [Open API v1 specification]({{< ref_static 
"generated/rest_v1_sql_gateway.yml" >}}) | 允许用户提交 SQL 语句到 Gateway 并执行。 |
-| [Open API v2 specification]({{< ref_static 
"generated/rest_v2_sql_gateway.yml" >}}) | 支持 SQL 客户端连接到 Gateway。 |
+| [Open API v2 specification]({{< ref_static 
"generated/rest_v2_sql_gateway.yml" >}}) | 支持 SQL 客户端连接到 Gateway。 |
+| [Open API v3 specification]({{< ref_static 
"generated/rest_v3_sql_gateway.yml" >}}) | 支持 Materialized Table 刷新操作。 |
 
 {{< hint warning >}}
 OpenAPI 规范目前仍处于实验阶段。
@@ -105,6 +106,11 @@ OpenAPI 规范目前仍处于实验阶段。
  API reference
 
 {{< tabs "f00ed142-b05f-44f0-bafc-799080c1d40d" >}}
+{{< tab "v3" >}}
+
+{{< generated/rest_v3_sql_gateway >}}
+
+{{< /tab >}}
 {{< tab "v2" >}}
 
 {{< generated/rest_v2_sql_gateway >}}
diff --git a/docs/content/docs/dev/table/sql-gateway/rest.md 
b/docs/content/docs/dev/table/sql-gateway/rest.md
index a38933f607c..4a4787f030c 100644
--- a/docs/content/docs/dev/table/sql-gateway/rest.md
+++ b/docs/content/docs/dev/table/sql-gateway/rest.md
@@ -100,6 +100,7 @@ The available OpenAPI specification is as follows. The 
default version is v2.
 | --- | --- |
 | [Open API v1 specification]({{< ref_static 
"generated/rest_v1_sql_gateway.yml" >}}) | Allow users to submit statements to 
the gateway and execute. |
 | [Open API v2 specification]({{< ref_static 
"generated/rest_v2_sql_gateway.yml" >}}) | Supports SQL Client to connect to 
the gateway. |
+| [Open API v3 specification]({{< ref_static 
"generated/rest_v3_sql_gateway.yml" >}}) | Supports Materialized Table refresh 
operation. |
 
 {{< hint warning >}}
 The OpenAPI specification is still experimental.
@@ -108,6 +109,11 @@ The OpenAPI specification is still experimental.
  API reference
 
 {{< tabs "f00ed142-b05f-44f0-bafc-799080c1d40d" >}}
+{{< tab "v3" >}}
+
+{{< generated/rest_v3_sql_gateway >}}
+
+{{< /tab >}}
 {{< tab "v2" >}}
 
 {{< generated/rest_v2_sql_gateway >}}
diff --git a/docs/hugo_extended_0.110.0_Linux-64bit.tar.gz 
b/docs/hugo_extended_0.110.0_Linux-64bit.tar.gz
new file mode 100644
index 000..61791d22a92
Binary files /dev/null and b/docs/hugo_extended_0.110.0_Linux-64bit.tar.gz 
differ
diff --git a/docs/layouts/shortcodes/generated/rest_v3_sql_gateway.html 
b/docs/layouts/shortcodes/generated/rest_v3_sql_gateway.html
new file mode 100644
index 000..8f7c6a7a022
--- /dev/null
+++ b/docs/layouts/shortcodes/generated/rest_v3_sql_gateway.html
@@ -0,0 +1,762 @@
+
+  
+
+  /api_versions
+
+
+  Verb: GET
+  Response code: 200 OK
+
+
+  Get the current available versions for the Rest 
Endpoint. The client can choose one of the return version as the protocol for 
later communicate.
+
+
+  
+
+  
+  Request
+  {}
+
+  
+
+
+  
+
+  
+  Response
+  {
+  "type" : "object",
+  "id" : 
"urn:jsonschema:org:apache:flink:table:gateway:rest:message:util:GetApiVersionResponseBody",
+  "properties" : {
+"versions" : {
+  "type" : "array",
+  "items" : {
+"type" : "string"
+  }
+}
+  }
+}
+
+  
+
+  
+
+
+  
+
+  /info
+
+
+  Verb: GET
+  Response code: 200 OK
+
+
+  Get meta data for

(flink) 01/03: [FLINK-35646][docs] Fix the error description of token parameter

2024-06-21 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit f386724029285864354abacafebbff725a0d0b9e
Author: fengli 
AuthorDate: Fri Jun 21 15:42:41 2024 +0800

[FLINK-35646][docs] Fix the error description of token parameter
---
 docs/layouts/shortcodes/generated/rest_v1_dispatcher.html  | 4 
 docs/layouts/shortcodes/generated/rest_v1_sql_gateway.html | 2 +-
 docs/layouts/shortcodes/generated/rest_v2_sql_gateway.html | 2 +-
 docs/static/generated/rest_v1_dispatcher.yml   | 7 +++
 docs/static/generated/rest_v1_sql_gateway.yml  | 2 +-
 docs/static/generated/rest_v2_sql_gateway.yml  | 2 +-
 .../rest/message/statement/FetchResultsTokenPathParameter.java | 2 +-
 7 files changed, 16 insertions(+), 5 deletions(-)

diff --git a/docs/layouts/shortcodes/generated/rest_v1_dispatcher.html 
b/docs/layouts/shortcodes/generated/rest_v1_dispatcher.html
index 7390bf744cd..6ecd4a32d8b 100644
--- a/docs/layouts/shortcodes/generated/rest_v1_dispatcher.html
+++ b/docs/layouts/shortcodes/generated/rest_v1_dispatcher.html
@@ -1182,6 +1182,10 @@ Using 'curl' you can upload a jar via 'curl -X POST -H 
"Expect:" -F "jarfile=@pa
 "jid" : {
   "type" : "any"
 },
+"job-type" : {
+  "type" : "string",
+  "enum" : [ "BATCH", "STREAMING" ]
+},
 "maxParallelism" : {
   "type" : "integer"
 },
diff --git a/docs/layouts/shortcodes/generated/rest_v1_sql_gateway.html 
b/docs/layouts/shortcodes/generated/rest_v1_sql_gateway.html
index 754657d3a2e..0fa86ac6cea 100644
--- a/docs/layouts/shortcodes/generated/rest_v1_sql_gateway.html
+++ b/docs/layouts/shortcodes/generated/rest_v1_sql_gateway.html
@@ -405,7 +405,7 @@
 
 session_handle - The SessionHandle that identifies a 
session.
 operation_handle - The OperationHandle that identifies a 
operation.
-token - The OperationHandle that identifies a operation.
+token - The token that identifies which batch of data to 
fetch.
 
   
 
diff --git a/docs/layouts/shortcodes/generated/rest_v2_sql_gateway.html 
b/docs/layouts/shortcodes/generated/rest_v2_sql_gateway.html
index 6f069491dd1..b64f83ba5c7 100644
--- a/docs/layouts/shortcodes/generated/rest_v2_sql_gateway.html
+++ b/docs/layouts/shortcodes/generated/rest_v2_sql_gateway.html
@@ -523,7 +523,7 @@ CREATE TABLE, DROP TABLE, ALTER TABLE, CREATE DATABASE, 
DROP DATABASE, ALTER DAT
 
 session_handle - The SessionHandle that identifies a 
session.
 operation_handle - The OperationHandle that identifies a 
operation.
-token - The OperationHandle that identifies a operation.
+token - The token that identifies which batch of data to 
fetch.
 
   
 
diff --git a/docs/static/generated/rest_v1_dispatcher.yml 
b/docs/static/generated/rest_v1_dispatcher.yml
index 94e1082085e..031b7484e3c 100644
--- a/docs/static/generated/rest_v1_dispatcher.yml
+++ b/docs/static/generated/rest_v1_dispatcher.yml
@@ -2395,6 +2395,8 @@ components:
   type: boolean
 jid:
   $ref: '#/components/schemas/JobID'
+job-type:
+  $ref: '#/components/schemas/JobType'
 maxParallelism:
   type: integer
   format: int64
@@ -2572,6 +2574,11 @@ components:
   properties:
 jobUrl:
   type: string
+JobType:
+  type: string
+  enum:
+  - BATCH
+  - STREAMING
 JobVertexAccumulatorsInfo:
   type: object
   properties:
diff --git a/docs/static/generated/rest_v1_sql_gateway.yml 
b/docs/static/generated/rest_v1_sql_gateway.yml
index b1572799799..5dc7104d666 100644
--- a/docs/static/generated/rest_v1_sql_gateway.yml
+++ b/docs/static/generated/rest_v1_sql_gateway.yml
@@ -166,7 +166,7 @@ paths:
   $ref: '#/components/schemas/OperationHandle'
   - name: token
 in: path
-description: The OperationHandle that identifies a operation.
+description: The token that identifies which batch of data to fetch.
 required: true
 schema:
   type: integer
diff --git a/docs/static/generated/rest_v2_sql_gateway.yml 
b/docs/static/generated/rest_v2_sql_gateway.yml
index c85b5734150..403863d62c4 100644
--- a/docs/static/generated/rest_v2_sql_gateway.yml
+++ b/docs/static/generated/rest_v2_sql_gateway.yml
@@ -210,7 +210,7 @@ paths:
   $ref: '#/components/schemas/OperationHandle'
   - name: token
 in: path
-description: The OperationHandle that identifies a operation.
+description: The token that identifies which batch of data to fetch.
 required: true
 schema:
   type: integer
diff --git 
a/flink-table/fl

(flink) branch master updated (b92a5944c42 -> 12f2222bbcd)

2024-06-21 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from b92a5944c42 [FLINK-35659][table] Remove unnecessary parameter 
customScheduleTime for embedded scheduler
 new f3867240292 [FLINK-35646][docs] Fix the error description of token 
parameter
 new 993c5c12df1 [FLINK-35646][table] Exclude embedded workflow scheduler 
related rest api from docs
 new 12fbbcd [FLINK-35646][docs] Add docs for materialized table 
refresh api

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 docs/content.zh/docs/dev/table/sql-gateway/rest.md |   8 +-
 docs/content/docs/dev/table/sql-gateway/rest.md|   6 ++
 docs/hugo_extended_0.110.0_Linux-64bit.tar.gz  | Bin 0 -> 601126 bytes
 .../shortcodes/generated/rest_v1_dispatcher.html   |   4 +
 .../shortcodes/generated/rest_v1_sql_gateway.html  |   2 +-
 .../shortcodes/generated/rest_v2_sql_gateway.html  |   2 +-
 ...2_sql_gateway.html => rest_v3_sql_gateway.html} |  85 -
 docs/static/generated/rest_v1_dispatcher.yml   |   7 ++
 docs/static/generated/rest_v1_sql_gateway.yml  |   2 +-
 docs/static/generated/rest_v2_sql_gateway.yml  |   2 +-
 ..._v2_sql_gateway.yml => rest_v3_sql_gateway.yml} |  59 +-
 .../RefreshMaterializedTableHeaders.java   |   5 ++
 .../CreateEmbeddedSchedulerWorkflowHeaders.java|   7 ++
 .../DeleteEmbeddedSchedulerWorkflowHeaders.java|   7 ++
 .../ResumeEmbeddedSchedulerWorkflowHeaders.java|   7 ++
 .../SuspendEmbeddedSchedulerWorkflowHeaders.java   |   7 ++
 .../MaterializedTableIdentifierPathParameter.java  |   2 +-
 .../statement/FetchResultsTokenPathParameter.java  |   2 +-
 18 files changed, 204 insertions(+), 10 deletions(-)
 create mode 100644 docs/hugo_extended_0.110.0_Linux-64bit.tar.gz
 copy docs/layouts/shortcodes/generated/{rest_v2_sql_gateway.html => 
rest_v3_sql_gateway.html} (88%)
 copy docs/static/generated/{rest_v2_sql_gateway.yml => 
rest_v3_sql_gateway.yml} (90%)



(flink) 01/02: [FLINK-35659][testutils] Fix TestFileSystem Connector error setting execution mode

2024-06-20 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 96fbb70546316403fe8cfc6816f27a846076ec6b
Author: fengli 
AuthorDate: Thu Jun 20 18:02:44 2024 +0800

[FLINK-35659][testutils] Fix TestFileSystem Connector error setting 
execution mode
---
 .../flink/table/factories/utils/FactoryMocks.java  |  8 +++-
 .../file/table/TestFileSystemTableSource.java  |  8 ++--
 .../file/testutils/TestFileSystemTableFactory.java |  6 +++
 .../testutils/TestFileSystemTableFactoryTest.java  | 44 ++
 4 files changed, 61 insertions(+), 5 deletions(-)

diff --git 
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/utils/FactoryMocks.java
 
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/utils/FactoryMocks.java
index 41e9a396a2b..0f9879f1044 100644
--- 
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/utils/FactoryMocks.java
+++ 
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/utils/FactoryMocks.java
@@ -19,6 +19,7 @@
 package org.apache.flink.table.factories.utils;
 
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.api.Schema;
 import org.apache.flink.table.catalog.CatalogTable;
@@ -55,6 +56,11 @@ public final class FactoryMocks {
 
 public static DynamicTableSource createTableSource(
 ResolvedSchema schema, Map options) {
+return createTableSource(schema, options, new Configuration());
+}
+
+public static DynamicTableSource createTableSource(
+ResolvedSchema schema, Map options, ReadableConfig 
readableConfig) {
 return FactoryUtil.createDynamicTableSource(
 null,
 IDENTIFIER,
@@ -66,7 +72,7 @@ public final class FactoryMocks {
 options),
 schema),
 Collections.emptyMap(),
-new Configuration(),
+readableConfig,
 FactoryMocks.class.getClassLoader(),
 false);
 }
diff --git 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/table/TestFileSystemTableSource.java
 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/table/TestFileSystemTableSource.java
index 3158f03473c..4a71fcdccab 100644
--- 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/table/TestFileSystemTableSource.java
+++ 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/table/TestFileSystemTableSource.java
@@ -19,9 +19,7 @@
 package org.apache.flink.connector.file.table;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.RuntimeExecutionMode;
 import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.configuration.ExecutionOptions;
 import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.connector.file.src.FileSourceSplit;
 import org.apache.flink.connector.file.src.TestFileSource;
@@ -42,11 +40,14 @@ import java.util.List;
 @Internal
 public class TestFileSystemTableSource extends FileSystemTableSource {
 
+private final boolean isStreamingMode;
+
 public TestFileSystemTableSource(
 ObjectIdentifier tableIdentifier,
 DataType physicalRowDataType,
 List partitionKeys,
 ReadableConfig tableOptions,
+boolean isStreamingMode,
 @Nullable DecodingFormat> 
bulkReaderFormat,
 @Nullable DecodingFormat> 
deserializationFormat) {
 super(
@@ -56,6 +57,7 @@ public class TestFileSystemTableSource extends 
FileSystemTableSource {
 tableOptions,
 bulkReaderFormat,
 deserializationFormat);
+this.isStreamingMode = isStreamingMode;
 }
 
 @Override
@@ -79,8 +81,6 @@ public class TestFileSystemTableSource extends 
FileSystemTableSource {
 new 
NonSplittingRecursiveAllDirEnumerator(
 regex)));
 
-boolean isStreamingMode =
-tableOptions.get(ExecutionOptions.RUNTIME_MODE) == 
RuntimeExecutionMode.STREAMING;
 fileSourceBuilder.setStreamingMode(isStreamingMode);
 
 return SourceProvider.of(fileSourceBuilder.build());
diff --git 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/TestFileSystemTableFactory.java
 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/mai

(flink) branch master updated (1f96cf8645a -> b92a5944c42)

2024-06-20 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 1f96cf8645a [FLINK-35658][runtime] Fix the issue that the external 
tier of hybrid shuffle can not work with an UnknownInputChannel
 new 96fbb705463 [FLINK-35659][testutils] Fix TestFileSystem Connector 
error setting execution mode
 new b92a5944c42 [FLINK-35659][table] Remove unnecessary parameter 
customScheduleTime for embedded scheduler

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../CreateEmbeddedSchedulerWorkflowHandler.java|  2 -
 ...CreateEmbeddedSchedulerWorkflowRequestBody.java | 13 +--
 .../MaterializedTableManager.java  | 10 +++--
 .../workflow/EmbeddedWorkflowScheduler.java|  1 -
 .../flink/table/gateway/workflow/WorkflowInfo.java | 16 
 .../scheduler/EmbeddedQuartzScheduler.java |  1 -
 .../workflow/EmbeddedSchedulerRelatedITCase.java   |  1 -
 .../gateway/workflow/QuartzSchedulerUtilsTest.java |  4 +-
 .../resources/sql_gateway_rest_api_v3.snapshot |  3 --
 .../flink/table/factories/utils/FactoryMocks.java  |  8 +++-
 .../file/table/TestFileSystemTableSource.java  |  8 ++--
 .../file/testutils/TestFileSystemTableFactory.java |  6 +++
 .../testutils/TestFileSystemTableFactoryTest.java  | 44 ++
 13 files changed, 69 insertions(+), 48 deletions(-)



(flink) 02/02: [FLINK-35659][table] Remove unnecessary parameter customScheduleTime for embedded scheduler

2024-06-20 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit b92a5944c42a66f88ae63127ae1ff1483a8c40fd
Author: fengli 
AuthorDate: Thu Jun 20 19:39:14 2024 +0800

[FLINK-35659][table] Remove unnecessary parameter customScheduleTime for 
embedded scheduler
---
 .../CreateEmbeddedSchedulerWorkflowHandler.java  |  2 --
 .../CreateEmbeddedSchedulerWorkflowRequestBody.java  | 13 +
 .../materializedtable/MaterializedTableManager.java  | 10 ++
 .../gateway/workflow/EmbeddedWorkflowScheduler.java  |  1 -
 .../flink/table/gateway/workflow/WorkflowInfo.java   | 16 
 .../workflow/scheduler/EmbeddedQuartzScheduler.java  |  1 -
 .../gateway/workflow/EmbeddedSchedulerRelatedITCase.java |  1 -
 .../table/gateway/workflow/QuartzSchedulerUtilsTest.java |  4 +---
 .../src/test/resources/sql_gateway_rest_api_v3.snapshot  |  3 ---
 9 files changed, 8 insertions(+), 43 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowHandler.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowHandler.java
index 9a7071ab935..61f96131496 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowHandler.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowHandler.java
@@ -74,7 +74,6 @@ public class CreateEmbeddedSchedulerWorkflowHandler
 String cronExpression = request.getRequestBody().getCronExpression();
 Map initConfig = 
request.getRequestBody().getInitConfig();
 Map executionConfig = 
request.getRequestBody().getExecutionConfig();
-String customScheduleTime = 
request.getRequestBody().getCustomScheduleTime();
 String restEndpointURL = request.getRequestBody().getRestEndpointUrl();
 WorkflowInfo workflowInfo =
 new WorkflowInfo(
@@ -82,7 +81,6 @@ public class CreateEmbeddedSchedulerWorkflowHandler
 Collections.emptyMap(),
 initConfig == null ? Collections.emptyMap() : 
initConfig,
 executionConfig == null ? Collections.emptyMap() : 
executionConfig,
-customScheduleTime,
 restEndpointURL);
 try {
 JobDetail jobDetail =
diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/message/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowRequestBody.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/message/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowRequestBody.java
index d0ebf3201ba..03b48e9417a 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/message/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowRequestBody.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/message/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowRequestBody.java
@@ -36,7 +36,6 @@ public class CreateEmbeddedSchedulerWorkflowRequestBody 
implements RequestBody {
 private static final String FIELD_NAME_CRON_EXPRESSION = "cronExpression";
 private static final String FIELD_NAME_INIT_CONFIG = "initConfig";
 private static final String FIELD_NAME_EXECUTION_CONFIG = 
"executionConfig";
-private static final String FIELD_NAME_SCHEDULE_TIME = 
"customScheduleTime";
 private static final String FIELD_NAME_REST_ENDPOINT_URL = 
"restEndpointUrl";
 
 @JsonProperty(FIELD_NAME_MATERIALIZED_TABLE)
@@ -52,10 +51,7 @@ public class CreateEmbeddedSchedulerWorkflowRequestBody 
implements RequestBody {
 @Nullable
 private final Map executionConfig;
 
-@JsonProperty(FIELD_NAME_SCHEDULE_TIME)
-@Nullable
-private final String customScheduleTime;
-
+@JsonProperty(FIELD_NAME_REST_ENDPOINT_URL)
 private final String restEndpointUrl;
 
 @JsonCreator
@@ -65,13 +61,11 @@ public class CreateEmbeddedSchedulerWorkflowRequestBody 
implements RequestBody {
 @Nullable @JsonProperty(FIELD_NAME_INIT_CONFIG) Map initConfig,
 @Nullable @JsonProperty(FIELD_NAME_EXECUTION_CONFIG)
 Map executionConfig,
-@Nullable @JsonProperty(FIELD_NAME_SCHEDULE_TIME) String 
customScheduleTime,
 @JsonProperty(FIELD_NAME_REST_ENDPOINT_URL) String 
restEndpointUrl) {
 this.materializedTableIdentifier = materializedTableIdentif

(flink) branch master updated (9708f9fd657 -> f462926ad9c)

2024-06-06 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 9708f9fd657 [FLINK-35501] Use common IO thread pool for RocksDB data 
transfer
 add 42289bd2c69 [FLINK-35201][table] Support the execution of drop 
materialized table in full refresh mode
 add c862fa60119 [FLINK-35201][table] Enhance function names in 
MaterializedTableStatementITCase for better readability
 add f462926ad9c [FLINK-35201][table] Remove unnecessary logs in 
MaterializedTableManager

No new revisions were added by this update.

Summary of changes:
 .../MaterializedTableManager.java  | 133 +++--
 .../scheduler/EmbeddedQuartzScheduler.java |   5 -
 .../AbstractMaterializedTableStatementITCase.java  |  23 ++--
 .../service/MaterializedTableStatementITCase.java  | 110 +++--
 .../workflow/EmbeddedSchedulerRelatedITCase.java   |  27 +
 5 files changed, 182 insertions(+), 116 deletions(-)



(flink) 02/03: [FLINK-35200][table] Support the execution of suspend, resume materialized table in full refresh mode

2024-06-04 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 9b51711d00a2e1bd93f5a474b9c99b542aaf27cf
Author: Feng Jin 
AuthorDate: Sat Jun 1 23:43:54 2024 +0800

[FLINK-35200][table] Support the execution of suspend, resume materialized 
table in full refresh mode

This closes #24877
---
 .../MaterializedTableManager.java  | 308 ++---
 .../AbstractMaterializedTableStatementITCase.java  |  12 +-
 ...GatewayRestEndpointMaterializedTableITCase.java |  10 +-
 .../service/MaterializedTableStatementITCase.java  | 130 -
 4 files changed, 337 insertions(+), 123 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
index eeb6b5109e3..ea2a56e2010 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
@@ -56,6 +56,9 @@ import 
org.apache.flink.table.refresh.RefreshHandlerSerializer;
 import org.apache.flink.table.types.logical.LogicalTypeFamily;
 import org.apache.flink.table.workflow.CreatePeriodicRefreshWorkflow;
 import org.apache.flink.table.workflow.CreateRefreshWorkflow;
+import org.apache.flink.table.workflow.ModifyRefreshWorkflow;
+import org.apache.flink.table.workflow.ResumeRefreshWorkflow;
+import org.apache.flink.table.workflow.SuspendRefreshWorkflow;
 import org.apache.flink.table.workflow.WorkflowScheduler;
 
 import org.slf4j.Logger;
@@ -173,10 +176,10 @@ public class MaterializedTableManager {
 CatalogMaterializedTable materializedTable =
 createMaterializedTableOperation.getCatalogMaterializedTable();
 if (CatalogMaterializedTable.RefreshMode.CONTINUOUS == 
materializedTable.getRefreshMode()) {
-createMaterializedInContinuousMode(
+createMaterializedTableInContinuousMode(
 operationExecutor, handle, 
createMaterializedTableOperation);
 } else {
-createMaterializedInFullMode(
+createMaterializedTableInFullMode(
 operationExecutor, handle, 
createMaterializedTableOperation);
 }
 // Just return ok for unify different refresh job info of continuous 
and full mode, user
@@ -184,7 +187,7 @@ public class MaterializedTableManager {
 return ResultFetcher.fromTableResult(handle, TABLE_RESULT_OK, false);
 }
 
-private void createMaterializedInContinuousMode(
+private void createMaterializedTableInContinuousMode(
 OperationExecutor operationExecutor,
 OperationHandle handle,
 CreateMaterializedTableOperation createMaterializedTableOperation) 
{
@@ -207,17 +210,21 @@ public class MaterializedTableManager {
 } catch (Exception e) {
 // drop materialized table while submit flink streaming job occur 
exception. Thus, weak
 // atomicity is guaranteed
-LOG.warn(
-"Submit continuous refresh job occur exception, drop 
materialized table {}.",
-materializedTableIdentifier,
-e);
 operationExecutor.callExecutableOperation(
 handle, new 
DropMaterializedTableOperation(materializedTableIdentifier, true));
-throw e;
+LOG.error(
+"Submit continuous refresh job for materialized table {} 
occur exception.",
+materializedTableIdentifier,
+e);
+throw new SqlExecutionException(
+String.format(
+"Submit continuous refresh job for materialized 
table %s occur exception.",
+materializedTableIdentifier),
+e);
 }
 }
 
-private void createMaterializedInFullMode(
+private void createMaterializedTableInFullMode(
 OperationExecutor operationExecutor,
 OperationHandle handle,
 CreateMaterializedTableOperation createMaterializedTableOperation) 
{
@@ -258,12 +265,13 @@ public class MaterializedTableManager {
 handle,
 materializedTableIdentifier,
 catalogMaterializedTable,
+CatalogMaterializedTable.RefreshStatus.ACTIVATED,
 refreshHandler.asSummaryString(),
 serializedRefreshHandler);
 } catch (Exception e) {
 // drop materialized table while create refresh workflo

(flink) branch master updated (62f9de806ac -> 2e158fe300f)

2024-06-04 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 62f9de806ac fixup! [FLINK-35351][checkpoint] Fix fail during restore 
from unaligned checkpoint with custom partitioner
 new 8d1e043b0c4 [FLINK-35200][table] Add dynamic options for 
ResumeRefreshWorkflow
 new 9b51711d00a [FLINK-35200][table] Support the execution of suspend, 
resume materialized table in full refresh mode
 new 2e158fe300f [FLINK-35200][table] Fix missing clusterInfo in 
materialized table refresh rest API return value

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../ResumeEmbeddedSchedulerWorkflowHandler.java|  17 +-
 .../ResumeEmbeddedSchedulerWorkflowHeaders.java|  42 ++-
 ...esumeEmbeddedSchedulerWorkflowRequestBody.java} |  24 +-
 .../MaterializedTableManager.java  | 341 +++--
 .../table/gateway/service/utils/Constants.java |   1 +
 .../workflow/EmbeddedWorkflowScheduler.java|  17 +-
 .../scheduler/EmbeddedQuartzScheduler.java |  50 ++-
 .../AbstractMaterializedTableStatementITCase.java  |  12 +-
 ...GatewayRestEndpointMaterializedTableITCase.java |  96 --
 .../service/MaterializedTableStatementITCase.java  | 130 +++-
 .../workflow/EmbeddedSchedulerRelatedITCase.java   |  14 +-
 .../resources/sql_gateway_rest_api_v3.snapshot |   8 +-
 .../table/workflow/ResumeRefreshWorkflow.java  |  11 +-
 13 files changed, 601 insertions(+), 162 deletions(-)
 copy 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/message/materializedtable/scheduler/{EmbeddedSchedulerWorkflowRequestBody.java
 => ResumeEmbeddedSchedulerWorkflowRequestBody.java} (73%)



(flink) 01/03: [FLINK-35200][table] Add dynamic options for ResumeRefreshWorkflow

2024-06-04 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 8d1e043b0c4277582b8862c2bc3314631eec4a7b
Author: Feng Jin 
AuthorDate: Sat Jun 1 23:43:07 2024 +0800

[FLINK-35200][table] Add dynamic options for ResumeRefreshWorkflow

This closes #24877
---
 .../ResumeEmbeddedSchedulerWorkflowHandler.java| 17 --
 .../ResumeEmbeddedSchedulerWorkflowHeaders.java| 42 -
 ...ResumeEmbeddedSchedulerWorkflowRequestBody.java | 71 ++
 .../workflow/EmbeddedWorkflowScheduler.java| 17 --
 .../scheduler/EmbeddedQuartzScheduler.java | 50 ++-
 .../workflow/EmbeddedSchedulerRelatedITCase.java   | 14 -
 .../resources/sql_gateway_rest_api_v3.snapshot |  8 ++-
 .../table/workflow/ResumeRefreshWorkflow.java  | 11 +++-
 8 files changed, 212 insertions(+), 18 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/scheduler/ResumeEmbeddedSchedulerWorkflowHandler.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/scheduler/ResumeEmbeddedSchedulerWorkflowHandler.java
index 4d0979946b8..d5030367839 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/scheduler/ResumeEmbeddedSchedulerWorkflowHandler.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/scheduler/ResumeEmbeddedSchedulerWorkflowHandler.java
@@ -25,7 +25,7 @@ import 
org.apache.flink.runtime.rest.messages.EmptyResponseBody;
 import org.apache.flink.runtime.rest.messages.MessageHeaders;
 import org.apache.flink.table.gateway.api.SqlGatewayService;
 import 
org.apache.flink.table.gateway.rest.handler.AbstractSqlGatewayRestHandler;
-import 
org.apache.flink.table.gateway.rest.message.materializedtable.scheduler.EmbeddedSchedulerWorkflowRequestBody;
+import 
org.apache.flink.table.gateway.rest.message.materializedtable.scheduler.ResumeEmbeddedSchedulerWorkflowRequestBody;
 import org.apache.flink.table.gateway.rest.util.SqlGatewayRestAPIVersion;
 import 
org.apache.flink.table.gateway.workflow.scheduler.EmbeddedQuartzScheduler;
 
@@ -34,13 +34,16 @@ import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseSt
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
+import java.util.Collections;
 import java.util.Map;
 import java.util.concurrent.CompletableFuture;
 
 /** Handler to resume workflow in embedded scheduler. */
 public class ResumeEmbeddedSchedulerWorkflowHandler
 extends AbstractSqlGatewayRestHandler<
-EmbeddedSchedulerWorkflowRequestBody, EmptyResponseBody, 
EmptyMessageParameters> {
+ResumeEmbeddedSchedulerWorkflowRequestBody,
+EmptyResponseBody,
+EmptyMessageParameters> {
 
 private final EmbeddedQuartzScheduler quartzScheduler;
 
@@ -49,7 +52,7 @@ public class ResumeEmbeddedSchedulerWorkflowHandler
 EmbeddedQuartzScheduler quartzScheduler,
 Map responseHeaders,
 MessageHeaders<
-EmbeddedSchedulerWorkflowRequestBody,
+ResumeEmbeddedSchedulerWorkflowRequestBody,
 EmptyResponseBody,
 EmptyMessageParameters>
 messageHeaders) {
@@ -60,12 +63,16 @@ public class ResumeEmbeddedSchedulerWorkflowHandler
 @Override
 protected CompletableFuture handleRequest(
 @Nullable SqlGatewayRestAPIVersion version,
-@Nonnull HandlerRequest 
request)
+@Nonnull 
HandlerRequest request)
 throws RestHandlerException {
 String workflowName = request.getRequestBody().getWorkflowName();
 String workflowGroup = request.getRequestBody().getWorkflowGroup();
+Map dynamicOptions = 
request.getRequestBody().getDynamicOptions();
 try {
-quartzScheduler.resumeScheduleWorkflow(workflowName, 
workflowGroup);
+quartzScheduler.resumeScheduleWorkflow(
+workflowName,
+workflowGroup,
+dynamicOptions == null ? Collections.emptyMap() : 
dynamicOptions);
 return 
CompletableFuture.completedFuture(EmptyResponseBody.getInstance());
 } catch (Exception e) {
 throw new RestHandlerException(
diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/ResumeEmbeddedSchedulerWorkflowHeaders.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/ResumeEmbeddedSchedulerWorkflowHeaders.java
index dface14468c..cc5

(flink) 03/03: [FLINK-35200][table] Fix missing clusterInfo in materialized table refresh rest API return value

2024-06-04 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 2e158fe300f6e93ba9b3d600e0237237ac0b2131
Author: Feng Jin 
AuthorDate: Tue Jun 4 01:56:08 2024 +0800

[FLINK-35200][table] Fix missing clusterInfo in materialized table refresh 
rest API return value

This closes #24877
---
 .../MaterializedTableManager.java  | 35 -
 .../table/gateway/service/utils/Constants.java |  1 +
 ...GatewayRestEndpointMaterializedTableITCase.java | 86 ++
 3 files changed, 104 insertions(+), 18 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
index ea2a56e2010..4c35e211e0d 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
@@ -23,15 +23,20 @@ import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.JobStatus;
 import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.api.config.TableConfigOptions;
 import org.apache.flink.table.catalog.CatalogMaterializedTable;
+import org.apache.flink.table.catalog.Column;
 import org.apache.flink.table.catalog.ObjectIdentifier;
 import org.apache.flink.table.catalog.ResolvedCatalogBaseTable;
 import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable;
 import org.apache.flink.table.catalog.ResolvedSchema;
 import org.apache.flink.table.catalog.TableChange;
+import org.apache.flink.table.data.GenericMapData;
+import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
 import org.apache.flink.table.factories.WorkflowSchedulerFactoryUtil;
 import org.apache.flink.table.gateway.api.operation.OperationHandle;
 import org.apache.flink.table.gateway.api.results.ResultSet;
@@ -94,6 +99,8 @@ import static 
org.apache.flink.table.api.internal.TableResultInternal.TABLE_RESU
 import static 
org.apache.flink.table.catalog.CatalogBaseTable.TableKind.MATERIALIZED_TABLE;
 import static 
org.apache.flink.table.factories.WorkflowSchedulerFactoryUtil.WORKFLOW_SCHEDULER_PREFIX;
 import static 
org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpointFactoryUtils.getEndpointConfig;
+import static 
org.apache.flink.table.gateway.service.utils.Constants.CLUSTER_INFO;
+import static org.apache.flink.table.gateway.service.utils.Constants.JOB_ID;
 import static org.apache.flink.table.utils.DateTimeUtils.formatTimestampString;
 import static 
org.apache.flink.table.utils.IntervalFreshnessUtils.convertFreshnessToCron;
 
@@ -594,11 +601,33 @@ public class MaterializedTableManager {
 dynamicOptions);
 
 try {
-LOG.debug(
-"Begin to manually refreshing the materialization table 
{}, statement: {}",
+LOG.info(
+"Begin to manually refreshing the materialized table {}, 
statement: {}",
 materializedTableIdentifier,
 insertStatement);
-return operationExecutor.executeStatement(handle, customConfig, 
insertStatement);
+ResultFetcher resultFetcher =
+operationExecutor.executeStatement(handle, customConfig, 
insertStatement);
+
+List results = fetchAllResults(resultFetcher);
+String jobId = results.get(0).getString(0).toString();
+String executeTarget =
+
operationExecutor.getSessionContext().getSessionConf().get(TARGET);
+Map clusterInfo = new HashMap<>();
+clusterInfo.put(
+StringData.fromString(TARGET.key()), 
StringData.fromString(executeTarget));
+// TODO get clusterId
+
+return ResultFetcher.fromResults(
+handle,
+ResolvedSchema.of(
+Column.physical(JOB_ID, DataTypes.STRING()),
+Column.physical(
+CLUSTER_INFO,
+DataTypes.MAP(DataTypes.STRING(), 
DataTypes.STRING(,
+Collections.singletonList(
+GenericRowData.of(
+StringData.fromString(jobId),
+new 

(flink) branch master updated (f3a3f926c6c -> e4fa72d9e48)

2024-06-01 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from f3a3f926c6c [FLINK-35483][runtime] Fix unstable BatchJobRecoveryTest.
 new 309e3246e02 [FLINK-35199][table] Remove dynamic options and add 
initialization configuration to CreatePeriodicRefreshWorkflow
 new e4fa72d9e48 [FLINK-35199][table] Support the execution of create 
materialized table in full refresh mode

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../table/client/gateway/SingleSessionManager.java |   1 +
 .../table/gateway/rest/SqlGatewayRestEndpoint.java |   6 +
 .../CreateEmbeddedSchedulerWorkflowHandler.java|   5 +-
 ...CreateEmbeddedSchedulerWorkflowRequestBody.java |  15 +-
 .../gateway/service/context/SessionContext.java|  36 ++-
 .../MaterializedTableManager.java  | 248 +
 .../service/operation/OperationExecutor.java   |  27 ++-
 .../table/gateway/service/session/Session.java |   4 +
 .../service/session/SessionManagerImpl.java|   1 +
 .../workflow/EmbeddedWorkflowScheduler.java|   2 +-
 .../flink/table/gateway/workflow/WorkflowInfo.java |  12 +-
 .../scheduler/EmbeddedQuartzScheduler.java | 248 -
 .../AbstractMaterializedTableStatementITCase.java  |  47 +++-
 ...GatewayRestEndpointMaterializedTableITCase.java |   8 -
 .../rest/util/SqlGatewayRestEndpointExtension.java |   4 +
 .../service/MaterializedTableStatementITCase.java  | 106 +++--
 .../gateway/workflow/QuartzSchedulerUtilsTest.java |  11 +-
 .../resources/sql_gateway_rest_api_v3.snapshot |   2 +-
 .../workflow/CreatePeriodicRefreshWorkflow.java|  10 +-
 19 files changed, 679 insertions(+), 114 deletions(-)



(flink) 01/02: [FLINK-35199][table] Remove dynamic options and add initialization configuration to CreatePeriodicRefreshWorkflow

2024-06-01 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 309e3246e0232a0a363aa44ab6d5524133f8f548
Author: Feng Jin 
AuthorDate: Fri May 31 11:41:12 2024 +0800

[FLINK-35199][table] Remove dynamic options and add initialization 
configuration to CreatePeriodicRefreshWorkflow
---
 .../scheduler/CreateEmbeddedSchedulerWorkflowHandler.java |  5 +++--
 .../CreateEmbeddedSchedulerWorkflowRequestBody.java   | 15 +++
 .../table/gateway/workflow/EmbeddedWorkflowScheduler.java |  2 +-
 .../apache/flink/table/gateway/workflow/WorkflowInfo.java | 12 +++-
 .../table/gateway/workflow/QuartzSchedulerUtilsTest.java  | 11 ---
 .../src/test/resources/sql_gateway_rest_api_v3.snapshot   |  2 +-
 .../table/workflow/CreatePeriodicRefreshWorkflow.java | 10 +-
 7 files changed, 36 insertions(+), 21 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowHandler.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowHandler.java
index b52094a39e6..9a7071ab935 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowHandler.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowHandler.java
@@ -72,14 +72,15 @@ public class CreateEmbeddedSchedulerWorkflowHandler
 String materializedTableIdentifier =
 request.getRequestBody().getMaterializedTableIdentifier();
 String cronExpression = request.getRequestBody().getCronExpression();
-Map dynamicOptions = 
request.getRequestBody().getDynamicOptions();
+Map initConfig = 
request.getRequestBody().getInitConfig();
 Map executionConfig = 
request.getRequestBody().getExecutionConfig();
 String customScheduleTime = 
request.getRequestBody().getCustomScheduleTime();
 String restEndpointURL = request.getRequestBody().getRestEndpointUrl();
 WorkflowInfo workflowInfo =
 new WorkflowInfo(
 materializedTableIdentifier,
-dynamicOptions == null ? Collections.emptyMap() : 
dynamicOptions,
+Collections.emptyMap(),
+initConfig == null ? Collections.emptyMap() : 
initConfig,
 executionConfig == null ? Collections.emptyMap() : 
executionConfig,
 customScheduleTime,
 restEndpointURL);
diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/message/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowRequestBody.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/message/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowRequestBody.java
index e0628933560..d0ebf3201ba 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/message/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowRequestBody.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/message/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowRequestBody.java
@@ -34,7 +34,7 @@ public class CreateEmbeddedSchedulerWorkflowRequestBody 
implements RequestBody {
 
 private static final String FIELD_NAME_MATERIALIZED_TABLE = 
"materializedTableIdentifier";
 private static final String FIELD_NAME_CRON_EXPRESSION = "cronExpression";
-private static final String FIELD_NAME_DYNAMIC_OPTIONS = "dynamicOptions";
+private static final String FIELD_NAME_INIT_CONFIG = "initConfig";
 private static final String FIELD_NAME_EXECUTION_CONFIG = 
"executionConfig";
 private static final String FIELD_NAME_SCHEDULE_TIME = 
"customScheduleTime";
 private static final String FIELD_NAME_REST_ENDPOINT_URL = 
"restEndpointUrl";
@@ -45,9 +45,8 @@ public class CreateEmbeddedSchedulerWorkflowRequestBody 
implements RequestBody {
 @JsonProperty(FIELD_NAME_CRON_EXPRESSION)
 private final String cronExpression;
 
-@JsonProperty(FIELD_NAME_DYNAMIC_OPTIONS)
-@Nullable
-private final Map dynamicOptions;
+@JsonProperty(FIELD_NAME_INIT_CONFIG)
+private final Map initConfig;
 
 @JsonProperty(FIELD_NAME_EXECUTION_CONFIG)
 @Nullable
@@ -63,14 +62,14 @@ public class CreateEmbeddedSchedulerWorkflowRequestBody 
implements RequestBody {
 public CreateEmbeddedSchedulerWorkflowRequestBody(
 @JsonProperty(FIELD_NAME_MATERIALIZED_

(flink) 02/02: [FLINK-35199][table] Support the execution of create materialized table in full refresh mode

2024-06-01 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit e4fa72d9e480664656818395741c37a9995f9334
Author: Feng Jin 
AuthorDate: Fri May 31 11:52:18 2024 +0800

[FLINK-35199][table] Support the execution of create materialized table in 
full refresh mode
---
 .../table/client/gateway/SingleSessionManager.java |   1 +
 .../table/gateway/rest/SqlGatewayRestEndpoint.java |   6 +
 .../gateway/service/context/SessionContext.java|  36 ++-
 .../MaterializedTableManager.java  | 248 +
 .../service/operation/OperationExecutor.java   |  27 ++-
 .../table/gateway/service/session/Session.java |   4 +
 .../service/session/SessionManagerImpl.java|   1 +
 .../scheduler/EmbeddedQuartzScheduler.java | 248 -
 .../AbstractMaterializedTableStatementITCase.java  |  47 +++-
 ...GatewayRestEndpointMaterializedTableITCase.java |   8 -
 .../rest/util/SqlGatewayRestEndpointExtension.java |   4 +
 .../service/MaterializedTableStatementITCase.java  | 106 +++--
 12 files changed, 643 insertions(+), 93 deletions(-)

diff --git 
a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/SingleSessionManager.java
 
b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/SingleSessionManager.java
index 27b1ccaa484..9c7e7dee0bb 100644
--- 
a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/SingleSessionManager.java
+++ 
b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/SingleSessionManager.java
@@ -96,6 +96,7 @@ public class SingleSessionManager implements SessionManager {
 sessionHandle,
 environment,
 operationExecutorService));
+session.open();
 return session;
 }
 
diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/SqlGatewayRestEndpoint.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/SqlGatewayRestEndpoint.java
index 2e24b967850..2fa462ade85 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/SqlGatewayRestEndpoint.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/SqlGatewayRestEndpoint.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.gateway.rest;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.rest.RestServerEndpoint;
@@ -83,6 +84,11 @@ public class SqlGatewayRestEndpoint extends 
RestServerEndpoint implements SqlGat
 quartzScheduler = new EmbeddedQuartzScheduler();
 }
 
+@VisibleForTesting
+public EmbeddedQuartzScheduler getQuartzScheduler() {
+return quartzScheduler;
+}
+
 @Override
 protected List> 
initializeHandlers(
 CompletableFuture localAddressFuture) {
diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java
index fa9ae05220f..cf1597ecea9 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java
@@ -38,6 +38,7 @@ import 
org.apache.flink.table.gateway.api.endpoint.EndpointVersion;
 import org.apache.flink.table.gateway.api.session.SessionEnvironment;
 import org.apache.flink.table.gateway.api.session.SessionHandle;
 import org.apache.flink.table.gateway.api.utils.SqlGatewayException;
+import 
org.apache.flink.table.gateway.service.materializedtable.MaterializedTableManager;
 import org.apache.flink.table.gateway.service.operation.OperationExecutor;
 import org.apache.flink.table.gateway.service.operation.OperationManager;
 import org.apache.flink.table.gateway.service.utils.SqlExecutionException;
@@ -237,6 +238,18 @@ public class SessionContext {
 statementSetOperations.add(operation);
 }
 
+public void open() {
+try {
+sessionState.materializedTableManager.open();
+} catch (Exception e) {
+LOG.error(
+String.format(
+"Failed to open the materialized table manager for 
the session %s.",
+sessionId),
+e);
+}
+}
+
 // 

 
 /** Close resources, e.g. catalogs. */
@@ -268,6 +281,15 @@ public class S

(flink) branch master updated (ce0b61f376b -> 2c35e48addf)

2024-05-29 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from ce0b61f376b [FLINK-35351][checkpoint] Clean up and unify code for the 
custom partitioner test case
 add bc14d551e04 [FLINK-35195][test/test-filesystem] test-filesystem 
support partition.fields option
 add 2c35e48addf [FLINK-35348][table] Introduce refresh materialized table 
rest api

No new revisions were added by this update.

Summary of changes:
 .../file/table/FileSystemTableFactory.java |   2 +-
 .../flink/table/gateway/api/SqlGatewayService.java |  28 ++
 .../gateway/api/utils/MockedSqlGatewayService.java |  14 +
 .../table/gateway/rest/SqlGatewayRestEndpoint.java |  15 +
 .../RefreshMaterializedTableHandler.java   |  95 
 .../RefreshMaterializedTableHeaders.java   |  96 
 .../MaterializedTableIdentifierPathParameter.java  |  46 ++
 .../RefreshMaterializedTableParameters.java|  56 +++
 .../RefreshMaterializedTableRequestBody.java   |  99 
 .../RefreshMaterializedTableResponseBody.java  |  43 ++
 .../gateway/service/SqlGatewayServiceImpl.java |  31 ++
 .../MaterializedTableManager.java  | 127 -
 .../service/operation/OperationExecutor.java   |  24 +
 .../AbstractMaterializedTableStatementITCase.java  | 339 +
 ...GatewayRestEndpointMaterializedTableITCase.java | 187 +++
 .../service/MaterializedTableStatementITCase.java  | 535 +++--
 .../MaterializedTableManagerTest.java  |  77 ++-
 .../resources/sql_gateway_rest_api_v3.snapshot |  57 +++
 .../api/config/MaterializedTableConfigOptions.java |   2 +
 .../file/testutils/TestFileSystemTableFactory.java |  16 +
 .../testutils/TestFileSystemTableFactoryTest.java  |   3 +
 21 files changed, 1602 insertions(+), 290 deletions(-)
 create mode 100644 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/RefreshMaterializedTableHandler.java
 create mode 100644 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/RefreshMaterializedTableHeaders.java
 create mode 100644 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/message/materializedtable/MaterializedTableIdentifierPathParameter.java
 create mode 100644 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/message/materializedtable/RefreshMaterializedTableParameters.java
 create mode 100644 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/message/materializedtable/RefreshMaterializedTableRequestBody.java
 create mode 100644 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/message/materializedtable/RefreshMaterializedTableResponseBody.java
 create mode 100644 
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractMaterializedTableStatementITCase.java
 create mode 100644 
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/SqlGatewayRestEndpointMaterializedTableITCase.java



(flink) 02/02: [FLINK-35425][table-common] Support convert freshness to cron expression in full refresh mode

2024-05-28 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 49f22254a78d554ac49810058c209297331129cd
Author: fengli 
AuthorDate: Mon May 27 20:54:39 2024 +0800

[FLINK-35425][table-common] Support convert freshness to cron expression in 
full refresh mode
---
 .../flink/table/utils/IntervalFreshnessUtils.java  | 74 
 .../table/utils/IntervalFreshnessUtilsTest.java| 80 +-
 .../SqlCreateMaterializedTableConverter.java   |  6 ++
 ...erializedTableNodeToOperationConverterTest.java |  9 +++
 4 files changed, 168 insertions(+), 1 deletion(-)

diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/IntervalFreshnessUtils.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/IntervalFreshnessUtils.java
index 121200098ec..cd58bff4d91 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/IntervalFreshnessUtils.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/IntervalFreshnessUtils.java
@@ -31,6 +31,15 @@ import java.time.Duration;
 @Internal
 public class IntervalFreshnessUtils {
 
+private static final String SECOND_CRON_EXPRESSION_TEMPLATE = "0/%s * * * 
* ? *";
+private static final String MINUTE_CRON_EXPRESSION_TEMPLATE = "0 0/%s * * 
* ? *";
+private static final String HOUR_CRON_EXPRESSION_TEMPLATE = "0 0 0/%s * * 
? *";
+private static final String ONE_DAY_CRON_EXPRESSION_TEMPLATE = "0 0 0 * * 
? *";
+
+private static final long SECOND_CRON_UPPER_BOUND = 60;
+private static final long MINUTE_CRON_UPPER_BOUND = 60;
+private static final long HOUR_CRON_UPPER_BOUND = 24;
+
 private IntervalFreshnessUtils() {}
 
 @VisibleForTesting
@@ -69,4 +78,69 @@ public class IntervalFreshnessUtils {
 intervalFreshness.getTimeUnit()));
 }
 }
+
+/**
+ * This is an util method that is used to convert the freshness of 
materialized table to cron
+ * expression in full refresh mode. Since freshness and cron expression 
cannot be converted
+ * equivalently, there are currently only a limited patterns of freshness 
that can be converted
+ * to cron expression.
+ */
+public static String convertFreshnessToCron(IntervalFreshness 
intervalFreshness) {
+switch (intervalFreshness.getTimeUnit()) {
+case SECOND:
+return validateAndConvertCron(
+intervalFreshness,
+SECOND_CRON_UPPER_BOUND,
+SECOND_CRON_EXPRESSION_TEMPLATE);
+case MINUTE:
+return validateAndConvertCron(
+intervalFreshness,
+MINUTE_CRON_UPPER_BOUND,
+MINUTE_CRON_EXPRESSION_TEMPLATE);
+case HOUR:
+return validateAndConvertCron(
+intervalFreshness, HOUR_CRON_UPPER_BOUND, 
HOUR_CRON_EXPRESSION_TEMPLATE);
+case DAY:
+return validateAndConvertDayCron(intervalFreshness);
+default:
+throw new ValidationException(
+String.format(
+"Unknown freshness time unit: %s.",
+intervalFreshness.getTimeUnit()));
+}
+}
+
+private static String validateAndConvertCron(
+IntervalFreshness intervalFreshness, long cronUpperBound, String 
cronTemplate) {
+long interval = Long.parseLong(intervalFreshness.getInterval());
+IntervalFreshness.TimeUnit timeUnit = intervalFreshness.getTimeUnit();
+// Freshness must be less than cronUpperBound for corresponding time 
unit when convert it
+// to cron expression
+if (interval >= cronUpperBound) {
+throw new ValidationException(
+String.format(
+"In full refresh mode, freshness must be less than 
%s when the time unit is %s.",
+cronUpperBound, timeUnit));
+}
+// Freshness must be factors of cronUpperBound for corresponding time 
unit
+if (cronUpperBound % interval != 0) {
+throw new ValidationException(
+String.format(
+"In full refresh mode, only freshness that are 
factors of %s are currently supported when the time unit is %s.",
+cronUpperBound, timeUnit));
+}
+
+return String.format(cronTemplate, interval);
+}
+
+private static String validateAndConvertDayCron(IntervalFreshness 
intervalFreshness) {
+// Since the number of days in each month is different, only one day 
of freshness is
+  

(flink) 01/02: [FLINK-35425][table-common] Introduce IntervalFreshness to support materialized table full refresh mode

2024-05-28 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 61a68bc9dc74926775dd546af64fe176782f70ba
Author: fengli 
AuthorDate: Fri May 24 12:24:49 2024 +0800

[FLINK-35425][table-common] Introduce IntervalFreshness to support 
materialized table full refresh mode
---
 .../catalog/CatalogBaseTableResolutionTest.java|  10 +-
 .../table/catalog/CatalogMaterializedTable.java|  19 +++-
 .../flink/table/catalog/CatalogPropertiesUtil.java |  20 +++-
 .../catalog/DefaultCatalogMaterializedTable.java   |   7 +-
 .../flink/table/catalog/IntervalFreshness.java | 104 +
 .../catalog/ResolvedCatalogMaterializedTable.java  |   5 +-
 .../flink/table/utils/IntervalFreshnessUtils.java  |  72 ++
 .../table/utils/IntervalFreshnessUtilsTest.java|  67 +
 .../SqlCreateMaterializedTableConverter.java   |   9 +-
 .../planner/utils/MaterializedTableUtils.java  |  16 ++--
 ...erializedTableNodeToOperationConverterTest.java |   4 +-
 .../catalog/TestFileSystemCatalogTest.java |   6 +-
 12 files changed, 302 insertions(+), 37 deletions(-)

diff --git 
a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/CatalogBaseTableResolutionTest.java
 
b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/CatalogBaseTableResolutionTest.java
index 72a22c22935..a9436ac21df 100644
--- 
a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/CatalogBaseTableResolutionTest.java
+++ 
b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/CatalogBaseTableResolutionTest.java
@@ -38,7 +38,6 @@ import org.junit.jupiter.api.Test;
 
 import javax.annotation.Nullable;
 
-import java.time.Duration;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
@@ -235,8 +234,8 @@ class CatalogBaseTableResolutionTest {
 
 assertThat(resolvedCatalogMaterializedTable.getResolvedSchema())
 .isEqualTo(RESOLVED_MATERIALIZED_TABLE_SCHEMA);
-assertThat(resolvedCatalogMaterializedTable.getFreshness())
-.isEqualTo(Duration.ofSeconds(30));
+assertThat(resolvedCatalogMaterializedTable.getDefinitionFreshness())
+.isEqualTo(IntervalFreshness.ofSecond("30"));
 assertThat(resolvedCatalogMaterializedTable.getDefinitionQuery())
 .isEqualTo(DEFINITION_QUERY);
 assertThat(resolvedCatalogMaterializedTable.getLogicalRefreshMode())
@@ -424,7 +423,8 @@ class CatalogBaseTableResolutionTest {
 properties.put("schema.3.comment", "");
 properties.put("schema.primary-key.name", "primary_constraint");
 properties.put("schema.primary-key.columns", "id");
-properties.put("freshness", "PT30S");
+properties.put("freshness-interval", "30");
+properties.put("freshness-unit", "SECOND");
 properties.put("logical-refresh-mode", "CONTINUOUS");
 properties.put("refresh-mode", "CONTINUOUS");
 properties.put("refresh-status", "INITIALIZING");
@@ -454,7 +454,7 @@ class CatalogBaseTableResolutionTest {
 .partitionKeys(partitionKeys)
 .options(Collections.emptyMap())
 .definitionQuery(definitionQuery)
-.freshness(Duration.ofSeconds(30))
+.freshness(IntervalFreshness.ofSecond("30"))
 
.logicalRefreshMode(CatalogMaterializedTable.LogicalRefreshMode.AUTOMATIC)
 .refreshMode(CatalogMaterializedTable.RefreshMode.CONTINUOUS)
 
.refreshStatus(CatalogMaterializedTable.RefreshStatus.INITIALIZING)
diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java
index 51856cc859e..1b41ed0ddb9 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java
@@ -30,6 +30,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 
+import static 
org.apache.flink.table.utils.IntervalFreshnessUtils.convertFreshnessToDuration;
+
 /**
  * Represents the unresolved metadata of a materialized table in a {@link 
Catalog}.
  *
@@ -113,9 +115,18 @@ public interface CatalogMaterializedTable extends 
CatalogBaseTable {
 String getDefinitionQuery();
 
 /**
- * Get the freshness of materialized table which is used to determine the 
physical refresh mode.
+ 

(flink) branch master updated (6c417719972 -> 49f22254a78)

2024-05-28 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 6c417719972 [hotfix] Fix modification conflict between FLINK-35465 and 
FLINK-35359
 new 61a68bc9dc7 [FLINK-35425][table-common] Introduce IntervalFreshness to 
support materialized table full refresh mode
 new 49f22254a78 [FLINK-35425][table-common] Support convert freshness to 
cron expression in full refresh mode

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../catalog/CatalogBaseTableResolutionTest.java|  10 +-
 .../table/catalog/CatalogMaterializedTable.java|  19 ++-
 .../flink/table/catalog/CatalogPropertiesUtil.java |  20 ++-
 .../catalog/DefaultCatalogMaterializedTable.java   |   7 +-
 .../flink/table/catalog/IntervalFreshness.java | 104 +++
 .../catalog/ResolvedCatalogMaterializedTable.java  |   5 +-
 .../flink/table/utils/IntervalFreshnessUtils.java  | 146 +
 .../table/utils/IntervalFreshnessUtilsTest.java| 145 
 .../SqlCreateMaterializedTableConverter.java   |  15 ++-
 .../planner/utils/MaterializedTableUtils.java  |  16 ++-
 ...erializedTableNodeToOperationConverterTest.java |  13 +-
 .../catalog/TestFileSystemCatalogTest.java |   6 +-
 12 files changed, 469 insertions(+), 37 deletions(-)
 create mode 100644 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/IntervalFreshness.java
 create mode 100644 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/IntervalFreshnessUtils.java
 create mode 100644 
flink-table/flink-table-common/src/test/java/org/apache/flink/table/utils/IntervalFreshnessUtilsTest.java



(flink) branch master updated (4b342da6d14 -> 90e2d6cfeea)

2024-05-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 4b342da6d14 [FLINK-35426][table-planner] Change the distribution of 
DynamicFilteringDataCollector to Broadcast
 add 90e2d6cfeea [FLINK-35342][table] Fix the unstable 
MaterializedTableStatementITCase test due to wrong job status check logic

No new revisions were added by this update.

Summary of changes:
 .../gateway/service/MaterializedTableStatementITCase.java  | 10 ++
 1 file changed, 10 insertions(+)



(flink) branch master updated (0737220959f -> 71e6746727a)

2024-05-23 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 0737220959f [FLINK-35216] Support for RETURNING clause of JSON_QUERY
 add 0ec6302cff4 [FLINK-35347][table-common] Introduce RefreshWorkflow 
related implementation to support full refresh mode for materialized table
 add 62b8fee5208 [FLINK-35347][table] Introduce embedded scheduler to 
support full refresh mode for materialized table
 add 71e6746727a [FLINK-35347][table] Introduce EmbeddedWorkflowScheduler 
plugin based on embedded scheduler

No new revisions were added by this update.

Summary of changes:
 flink-table/flink-sql-gateway/pom.xml  |  26 ++
 .../table/gateway/rest/SqlGatewayRestEndpoint.java |  60 ++-
 .../CreateEmbeddedSchedulerWorkflowHandler.java|  98 
 .../DeleteEmbeddedSchedulerWorkflowHandler.java|  75 +++
 .../ResumeEmbeddedSchedulerWorkflowHandler.java|  75 +++
 .../SuspendEmbeddedSchedulerWorkflowHandler.java   |  75 +++
 .../AbstractEmbeddedSchedulerWorkflowHeaders.java  |  63 +++
 .../CreateEmbeddedSchedulerWorkflowHeaders.java}   |  65 ++-
 .../DeleteEmbeddedSchedulerWorkflowHeaders.java|  50 ++
 .../ResumeEmbeddedSchedulerWorkflowHeaders.java|  50 ++
 .../SuspendEmbeddedSchedulerWorkflowHeaders.java   |  50 ++
 .../header/session/ConfigureSessionHeaders.java|   4 +-
 .../header/statement/CompleteStatementHeaders.java |   4 +-
 ...CreateEmbeddedSchedulerWorkflowRequestBody.java | 105 +
 ...reateEmbeddedSchedulerWorkflowResponseBody.java |  53 +++
 .../EmbeddedSchedulerWorkflowRequestBody.java  |  55 +++
 .../rest/util/SqlGatewayRestAPIVersion.java|   5 +-
 .../gateway/workflow/EmbeddedRefreshHandler.java   |  84 
 .../workflow/EmbeddedRefreshHandlerSerializer.java |  45 ++
 .../workflow/EmbeddedWorkflowScheduler.java| 235 ++
 .../workflow/EmbeddedWorkflowSchedulerFactory.java |  67 +++
 .../flink/table/gateway/workflow/WorkflowInfo.java | 125 +
 .../scheduler/EmbeddedQuartzScheduler.java | 229 +
 .../workflow/scheduler/QuartzSchedulerUtils.java   | 125 +
 .../workflow/scheduler/SchedulerException.java}|  14 +-
 .../src/main/resources/META-INF/NOTICE |   9 +
 .../org.apache.flink.table.factories.Factory   |   1 +
 .../table/gateway/rest/RestAPIITCaseBase.java  |   6 +-
 .../rest/util/TestingSqlGatewayRestEndpoint.java   |   4 +-
 .../workflow/EmbeddedRefreshHandlerTest.java}  |  28 +-
 .../workflow/EmbeddedSchedulerRelatedITCase.java   | 350 ++
 .../gateway/workflow/QuartzSchedulerUtilsTest.java |  83 
 .../resources/sql_gateway_rest_api_v3.snapshot | 519 +
 .../table/refresh/ContinuousRefreshHandler.java|   2 +
 .../workflow/CreatePeriodicRefreshWorkflow.java|  85 
 ...owException.java => ResumeRefreshWorkflow.java} |  19 +-
 ...wException.java => SuspendRefreshWorkflow.java} |  19 +-
 .../flink/table/workflow/WorkflowException.java|   5 +-
 flink-table/pom.xml|   1 +
 39 files changed, 2887 insertions(+), 81 deletions(-)
 create mode 100644 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowHandler.java
 create mode 100644 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/scheduler/DeleteEmbeddedSchedulerWorkflowHandler.java
 create mode 100644 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/scheduler/ResumeEmbeddedSchedulerWorkflowHandler.java
 create mode 100644 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/materializedtable/scheduler/SuspendEmbeddedSchedulerWorkflowHandler.java
 create mode 100644 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/AbstractEmbeddedSchedulerWorkflowHeaders.java
 copy 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/{statement/CompleteStatementHeaders.java
 => materializedtable/scheduler/CreateEmbeddedSchedulerWorkflowHeaders.java} 
(51%)
 create mode 100644 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/DeleteEmbeddedSchedulerWorkflowHeaders.java
 create mode 100644 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/ResumeEmbeddedSchedulerWorkflowHeaders.java
 create mode 100644 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/header/materializedtable/scheduler/SuspendEmbeddedSchedulerWorkflowHeaders.java
 create mode 100644 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/message/material

(flink) branch master updated: [FLINK-35346][table-common] Introduce workflow scheduler interface for materialized table

2024-05-16 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 1378979f02e [FLINK-35346][table-common] Introduce workflow scheduler 
interface for materialized table
1378979f02e is described below

commit 1378979f02eed55bbf3f91b08ec166d55b2c42a6
Author: Ron 
AuthorDate: Thu May 16 19:41:54 2024 +0800

[FLINK-35346][table-common] Introduce workflow scheduler interface for 
materialized table

[FLINK-35346][table-common] Introduce workflow scheduler interface for 
materialized table

This closes #24767
---
 .../apache/flink/table/factories/FactoryUtil.java  |   9 +-
 .../table/factories/WorkflowSchedulerFactory.java  |  56 +++
 .../factories/WorkflowSchedulerFactoryUtil.java| 156 ++
 .../table/workflow/CreateRefreshWorkflow.java  |  29 
 .../table/workflow/DeleteRefreshWorkflow.java  |  48 ++
 .../table/workflow/ModifyRefreshWorkflow.java  |  40 +
 .../flink/table/workflow/RefreshWorkflow.java  |  34 
 .../flink/table/workflow/WorkflowException.java|  37 +
 .../flink/table/workflow/WorkflowScheduler.java|  91 +++
 .../workflow/TestWorkflowSchedulerFactory.java | 175 +
 .../workflow/WorkflowSchedulerFactoryUtilTest.java | 107 +
 .../org.apache.flink.table.factories.Factory   |   1 +
 12 files changed, 782 insertions(+), 1 deletion(-)

diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java
index d8d6d7e9000..5d66b23c3d8 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java
@@ -167,6 +167,13 @@ public final class FactoryUtil {
 + "tasks to advance their watermarks 
without the need to wait for "
 + "watermarks from this source while it is 
idle.");
 
+public static final ConfigOption WORKFLOW_SCHEDULER_TYPE =
+ConfigOptions.key("workflow-scheduler.type")
+.stringType()
+.noDefaultValue()
+.withDescription(
+"Specify the workflow scheduler type that is used 
for materialized table.");
+
 /**
  * Suffix for keys of {@link ConfigOption} in case a connector requires 
multiple formats (e.g.
  * for both key and value).
@@ -903,7 +910,7 @@ public final class FactoryUtil {
 return loadResults;
 }
 
-private static String stringifyOption(String key, String value) {
+public static String stringifyOption(String key, String value) {
 if (GlobalConfiguration.isSensitive(key)) {
 value = HIDDEN_CONTENT;
 }
diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/WorkflowSchedulerFactory.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/WorkflowSchedulerFactory.java
new file mode 100644
index 000..72e144f7d19
--- /dev/null
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/WorkflowSchedulerFactory.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.table.factories;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.workflow.WorkflowScheduler;
+
+import java.util.Map;
+
+/**
+ * A factory to create a {@link WorkflowScheduler} instance.
+ *
+ * See {@link Factory} for more information about the general design of a 
factory.
+ */
+@PublicEvolving
+public interface WorkflowSchedulerFactory extends Factory {
+
+/** Create a workflow scheduler instance which interacts with external 
scheduler service. */
+ 

(flink) 01/04: [FLINK-35193][table] Support drop materialized table syntax

2024-05-14 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 8551ef39e0387f723a72299cc73aaaf827cf74bf
Author: Feng Jin 
AuthorDate: Mon May 13 20:06:41 2024 +0800

[FLINK-35193][table] Support drop materialized table syntax
---
 .../src/main/codegen/data/Parser.tdd   |  1 +
 .../src/main/codegen/includes/parserImpls.ftl  | 30 ++
 .../sql/parser/ddl/SqlDropMaterializedTable.java   | 68 ++
 .../flink/sql/parser/utils/ParserResource.java |  3 +
 .../MaterializedTableStatementParserTest.java  | 25 
 5 files changed, 127 insertions(+)

diff --git a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd 
b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd
index 81b3412954c..883b6aec1b2 100644
--- a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd
+++ b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd
@@ -76,6 +76,7 @@
 "org.apache.flink.sql.parser.ddl.SqlDropCatalog"
 "org.apache.flink.sql.parser.ddl.SqlDropDatabase"
 "org.apache.flink.sql.parser.ddl.SqlDropFunction"
+"org.apache.flink.sql.parser.ddl.SqlDropMaterializedTable"
 "org.apache.flink.sql.parser.ddl.SqlDropPartitions"
 
"org.apache.flink.sql.parser.ddl.SqlDropPartitions.AlterTableDropPartitionsContext"
 "org.apache.flink.sql.parser.ddl.SqlDropTable"
diff --git 
a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl 
b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl
index bdc97818914..b2a5ea02d0f 100644
--- a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl
+++ b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl
@@ -1801,6 +1801,34 @@ SqlCreate SqlCreateMaterializedTable(Span s, boolean 
replace, boolean isTemporar
 }
 }
 
+/**
+* Parses a DROP MATERIALIZED TABLE statement.
+*/
+SqlDrop SqlDropMaterializedTable(Span s, boolean replace, boolean isTemporary) 
:
+{
+SqlIdentifier tableName = null;
+boolean ifExists = false;
+}
+{
+
+ {
+ if (isTemporary) {
+ throw SqlUtil.newContextException(
+ getPos(),
+ 
ParserResource.RESOURCE.dropTemporaryMaterializedTableUnsupported());
+ }
+ }
+ 
+
+ifExists = IfExistsOpt()
+
+tableName = CompoundIdentifier()
+
+{
+return new SqlDropMaterializedTable(s.pos(), tableName, ifExists);
+}
+}
+
 /**
 * Parses alter materialized table.
 */
@@ -2427,6 +2455,8 @@ SqlDrop SqlDropExtended(Span s, boolean replace) :
 (
 drop = SqlDropCatalog(s, replace)
 |
+drop = SqlDropMaterializedTable(s, replace, isTemporary)
+|
 drop = SqlDropTable(s, replace, isTemporary)
 |
 drop = SqlDropView(s, replace, isTemporary)
diff --git 
a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlDropMaterializedTable.java
 
b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlDropMaterializedTable.java
new file mode 100644
index 000..ec9439fb13a
--- /dev/null
+++ 
b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlDropMaterializedTable.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.sql.parser.ddl;
+
+import org.apache.calcite.sql.SqlDrop;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.ImmutableNullableList;
+
+import java.util.List;
+
+/** DROP MATERIALIZED TABLE DDL sql call. */
+public class SqlDropMaterializedTable extends SqlDrop {
+
+private static final SqlOperator OPERATOR =
+new SqlSpecialOperator("DROP MATERIALIZED TABLE", 
SqlKind.DRO

(flink) 03/04: [FLINK-35193][table] Support execution of drop materialized table

2024-05-14 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 51b744bca1bdf53385152ed237f2950525046488
Author: Feng Jin 
AuthorDate: Mon May 13 20:08:38 2024 +0800

[FLINK-35193][table] Support execution of drop materialized table
---
 .../MaterializedTableManager.java  | 115 +-
 .../service/operation/OperationExecutor.java   |   9 +
 .../service/MaterializedTableStatementITCase.java  | 241 ++---
 .../apache/flink/table/catalog/CatalogManager.java |   4 +-
 4 files changed, 328 insertions(+), 41 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
index b4ba12b8755..a51b1885c98 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
@@ -20,6 +20,7 @@ package 
org.apache.flink.table.gateway.service.materializedtable;
 
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobStatus;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.catalog.CatalogMaterializedTable;
@@ -34,6 +35,7 @@ import org.apache.flink.table.gateway.api.results.ResultSet;
 import org.apache.flink.table.gateway.service.operation.OperationExecutor;
 import org.apache.flink.table.gateway.service.result.ResultFetcher;
 import org.apache.flink.table.gateway.service.utils.SqlExecutionException;
+import org.apache.flink.table.operations.command.DescribeJobOperation;
 import org.apache.flink.table.operations.command.StopJobOperation;
 import 
org.apache.flink.table.operations.materializedtable.AlterMaterializedTableChangeOperation;
 import 
org.apache.flink.table.operations.materializedtable.AlterMaterializedTableRefreshOperation;
@@ -93,6 +95,9 @@ public class MaterializedTableManager {
 } else if (op instanceof AlterMaterializedTableResumeOperation) {
 return callAlterMaterializedTableResume(
 operationExecutor, handle, 
(AlterMaterializedTableResumeOperation) op);
+} else if (op instanceof DropMaterializedTableOperation) {
+return callDropMaterializedTableOperation(
+operationExecutor, handle, 
(DropMaterializedTableOperation) op);
 }
 
 throw new SqlExecutionException(
@@ -146,8 +151,7 @@ public class MaterializedTableManager {
 materializedTableIdentifier,
 e);
 operationExecutor.callExecutableOperation(
-handle,
-new 
DropMaterializedTableOperation(materializedTableIdentifier, true, false));
+handle, new 
DropMaterializedTableOperation(materializedTableIdentifier, true));
 throw e;
 }
 }
@@ -170,7 +174,8 @@ public class MaterializedTableManager {
 materializedTable.getSerializedRefreshHandler(),
 
operationExecutor.getSessionContext().getUserClassloader());
 
-String savepointPath = stopJobWithSavepoint(operationExecutor, handle, 
refreshHandler);
+String savepointPath =
+stopJobWithSavepoint(operationExecutor, handle, 
refreshHandler.getJobId());
 
 ContinuousRefreshHandler updateRefreshHandler =
 new ContinuousRefreshHandler(
@@ -183,9 +188,12 @@ public class MaterializedTableManager {
 CatalogMaterializedTable.RefreshStatus.SUSPENDED,
 
materializedTable.getRefreshHandlerDescription().orElse(null),
 serializeContinuousHandler(updateRefreshHandler));
+List tableChanges = new ArrayList<>();
+tableChanges.add(
+
TableChange.modifyRefreshStatus(CatalogMaterializedTable.RefreshStatus.ACTIVATED));
 AlterMaterializedTableChangeOperation 
alterMaterializedTableChangeOperation =
 new AlterMaterializedTableChangeOperation(
-tableIdentifier, Collections.emptyList(), 
updatedMaterializedTable);
+tableIdentifier, tableChanges, 
updatedMaterializedTable);
 
 operationExecutor.callExecutableOperation(handle, 
alterMaterializedTableChangeOperation);
 
@@ -284,8 +292,7 @@ public class MaterializedTableManager {
 // drop materialized table while submit flink streaming job occur 
exception. Thus, weak
 // atomicity is guar

(flink) 04/04: [FLINK-35342][table] Fix MaterializedTableStatementITCase test can check for wrong status

2024-05-14 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 94d861b08fef1e350d80a3f5f0f63168d327bc64
Author: Feng Jin 
AuthorDate: Tue May 14 11:18:40 2024 +0800

[FLINK-35342][table] Fix MaterializedTableStatementITCase test can check 
for wrong status
---
 .../service/MaterializedTableStatementITCase.java| 20 +++-
 1 file changed, 15 insertions(+), 5 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
 
b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
index 105c51ea597..dd7d25e124c 100644
--- 
a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
+++ 
b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
@@ -272,7 +272,7 @@ public class MaterializedTableStatementITCase {
 waitUntilAllTasksAreRunning(
 restClusterClient, 
JobID.fromHexString(activeRefreshHandler.getJobId()));
 
-// check the background job is running
+// verify the background job is running
 String describeJobDDL = String.format("DESCRIBE JOB '%s'", 
activeRefreshHandler.getJobId());
 OperationHandle describeJobHandle =
 service.executeStatement(sessionHandle, describeJobDDL, -1, 
new Configuration());
@@ -653,7 +653,7 @@ public class MaterializedTableStatementITCase {
 assertThat(suspendMaterializedTable.getRefreshStatus())
 .isEqualTo(CatalogMaterializedTable.RefreshStatus.SUSPENDED);
 
-// check background job is stopped
+// verify background job is stopped
 byte[] refreshHandler = 
suspendMaterializedTable.getSerializedRefreshHandler();
 ContinuousRefreshHandler suspendRefreshHandler =
 ContinuousRefreshHandlerSerializer.INSTANCE.deserialize(
@@ -667,7 +667,7 @@ public class MaterializedTableStatementITCase {
 List jobResults = fetchAllResults(service, sessionHandle, 
describeJobHandle);
 
assertThat(jobResults.get(0).getString(2).toString()).isEqualTo("FINISHED");
 
-// check savepoint is created
+// verify savepoint is created
 assertThat(suspendRefreshHandler.getRestorePath()).isNotEmpty();
 String actualSavepointPath = 
suspendRefreshHandler.getRestorePath().get();
 
@@ -692,7 +692,17 @@ public class MaterializedTableStatementITCase {
 assertThat(resumedCatalogMaterializedTable.getRefreshStatus())
 .isEqualTo(CatalogMaterializedTable.RefreshStatus.ACTIVATED);
 
-// check background job is running
+waitUntilAllTasksAreRunning(
+restClusterClient,
+JobID.fromHexString(
+ContinuousRefreshHandlerSerializer.INSTANCE
+.deserialize(
+resumedCatalogMaterializedTable
+.getSerializedRefreshHandler(),
+getClass().getClassLoader())
+.getJobId()));
+
+// verify background job is running
 refreshHandler = 
resumedCatalogMaterializedTable.getSerializedRefreshHandler();
 ContinuousRefreshHandler resumeRefreshHandler =
 ContinuousRefreshHandlerSerializer.INSTANCE.deserialize(
@@ -706,7 +716,7 @@ public class MaterializedTableStatementITCase {
 jobResults = fetchAllResults(service, sessionHandle, 
describeResumeJobHandle);
 
assertThat(jobResults.get(0).getString(2).toString()).isEqualTo("RUNNING");
 
-// check resumed job is restored from savepoint
+// verify resumed job is restored from savepoint
 Optional actualRestorePath =
 getJobRestoreSavepointPath(restClusterClient, resumeJobId);
 assertThat(actualRestorePath).isNotEmpty();



(flink) 02/04: [FLINK-35193][table] Support convert drop materialized table node to operation

2024-05-14 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit fd333941553c68c36e1460102ab023f80a5b1362
Author: Feng Jin 
AuthorDate: Mon May 13 20:07:39 2024 +0800

[FLINK-35193][table] Support convert drop materialized table node to 
operation
---
 .../DropMaterializedTableOperation.java|  6 ++--
 .../SqlDropMaterializedTableConverter.java | 41 ++
 .../operations/converters/SqlNodeConverters.java   |  1 +
 ...erializedTableNodeToOperationConverterTest.java | 21 +++
 4 files changed, 65 insertions(+), 4 deletions(-)

diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/DropMaterializedTableOperation.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/DropMaterializedTableOperation.java
index e5eee557bfc..46dd86ad96b 100644
--- 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/DropMaterializedTableOperation.java
+++ 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/DropMaterializedTableOperation.java
@@ -33,9 +33,8 @@ import java.util.Map;
 public class DropMaterializedTableOperation extends DropTableOperation
 implements MaterializedTableOperation {
 
-public DropMaterializedTableOperation(
-ObjectIdentifier tableIdentifier, boolean ifExists, boolean 
isTemporary) {
-super(tableIdentifier, ifExists, isTemporary);
+public DropMaterializedTableOperation(ObjectIdentifier tableIdentifier, 
boolean ifExists) {
+super(tableIdentifier, ifExists, false);
 }
 
 @Override
@@ -43,7 +42,6 @@ public class DropMaterializedTableOperation extends 
DropTableOperation
 Map params = new LinkedHashMap<>();
 params.put("identifier", getTableIdentifier());
 params.put("IfExists", isIfExists());
-params.put("isTemporary", isTemporary());
 
 return OperationUtils.formatWithChildren(
 "DROP MATERIALIZED TABLE",
diff --git 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlDropMaterializedTableConverter.java
 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlDropMaterializedTableConverter.java
new file mode 100644
index 000..6501dc0c453
--- /dev/null
+++ 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlDropMaterializedTableConverter.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.operations.converters;
+
+import org.apache.flink.sql.parser.ddl.SqlDropMaterializedTable;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.UnresolvedIdentifier;
+import org.apache.flink.table.operations.Operation;
+import 
org.apache.flink.table.operations.materializedtable.DropMaterializedTableOperation;
+
+/** A converter for {@link SqlDropMaterializedTable}. */
+public class SqlDropMaterializedTableConverter
+implements SqlNodeConverter {
+@Override
+public Operation convertSqlNode(
+SqlDropMaterializedTable sqlDropMaterializedTable, ConvertContext 
context) {
+UnresolvedIdentifier unresolvedIdentifier =
+
UnresolvedIdentifier.of(sqlDropMaterializedTable.fullTableName());
+ObjectIdentifier identifier =
+
context.getCatalogManager().qualifyIdentifier(unresolvedIdentifier);
+// Currently we don't support temporary materialized table, so 
isTemporary is always false
+return new DropMaterializedTableOperation(
+identifier, sqlDropMaterializedTable.getIfExists());
+}
+}
diff --git 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConverters.java
 
b/flink-table/flink-table-pla

(flink) branch master updated (65d31e26534 -> 94d861b08fe)

2024-05-14 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 65d31e26534 [FLINK-33986][runtime] Extend ShuffleMaster to support 
snapshot and restore state.
 new 8551ef39e03 [FLINK-35193][table] Support drop materialized table syntax
 new fd333941553 [FLINK-35193][table] Support convert drop materialized 
table node to operation
 new 51b744bca1b [FLINK-35193][table] Support execution of drop 
materialized table
 new 94d861b08fe [FLINK-35342][table] Fix MaterializedTableStatementITCase 
test can check for wrong status

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../MaterializedTableManager.java  | 115 -
 .../service/operation/OperationExecutor.java   |   9 +
 .../service/MaterializedTableStatementITCase.java  | 261 ++---
 .../src/main/codegen/data/Parser.tdd   |   1 +
 .../src/main/codegen/includes/parserImpls.ftl  |  30 +++
 ...pCatalog.java => SqlDropMaterializedTable.java} |  40 ++--
 .../flink/sql/parser/utils/ParserResource.java |   3 +
 .../MaterializedTableStatementParserTest.java  |  25 ++
 .../apache/flink/table/catalog/CatalogManager.java |   4 +-
 .../DropMaterializedTableOperation.java|   6 +-
 ...java => SqlDropMaterializedTableConverter.java} |  20 +-
 .../operations/converters/SqlNodeConverters.java   |   1 +
 ...erializedTableNodeToOperationConverterTest.java |  21 ++
 13 files changed, 455 insertions(+), 81 deletions(-)
 copy 
flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/{SqlDropCatalog.java
 => SqlDropMaterializedTable.java} (68%)
 copy 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/{SqlAlterMaterializedTableSuspendConverter.java
 => SqlDropMaterializedTableConverter.java} (59%)



(flink) 01/01: [FLINK-35197][table] Support the execution of suspend, resume materialized table in continuous refresh mode

2024-05-12 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit e4972c003f68da6dc4066459d4c6e5d981f07e96
Author: Feng Jin 
AuthorDate: Thu May 9 16:26:12 2024 +0800

[FLINK-35197][table] Support the execution of suspend, resume materialized 
table in continuous refresh mode

This closes #24765
---
 .../MaterializedTableManager.java  | 215 ++-
 .../service/MaterializedTableStatementITCase.java  | 302 -
 .../MaterializedTableManagerTest.java  |  39 +++
 .../table/refresh/ContinuousRefreshHandler.java|  22 +-
 4 files changed, 561 insertions(+), 17 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
index ff0670462e0..b4ba12b8755 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
@@ -34,8 +34,11 @@ import org.apache.flink.table.gateway.api.results.ResultSet;
 import org.apache.flink.table.gateway.service.operation.OperationExecutor;
 import org.apache.flink.table.gateway.service.result.ResultFetcher;
 import org.apache.flink.table.gateway.service.utils.SqlExecutionException;
+import org.apache.flink.table.operations.command.StopJobOperation;
 import 
org.apache.flink.table.operations.materializedtable.AlterMaterializedTableChangeOperation;
 import 
org.apache.flink.table.operations.materializedtable.AlterMaterializedTableRefreshOperation;
+import 
org.apache.flink.table.operations.materializedtable.AlterMaterializedTableResumeOperation;
+import 
org.apache.flink.table.operations.materializedtable.AlterMaterializedTableSuspendOperation;
 import 
org.apache.flink.table.operations.materializedtable.CreateMaterializedTableOperation;
 import 
org.apache.flink.table.operations.materializedtable.DropMaterializedTableOperation;
 import 
org.apache.flink.table.operations.materializedtable.MaterializedTableOperation;
@@ -46,17 +49,23 @@ import 
org.apache.flink.table.types.logical.LogicalTypeFamily;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import static org.apache.flink.api.common.RuntimeExecutionMode.BATCH;
 import static org.apache.flink.api.common.RuntimeExecutionMode.STREAMING;
+import static 
org.apache.flink.configuration.CheckpointingOptions.SAVEPOINT_DIRECTORY;
 import static org.apache.flink.configuration.DeploymentOptions.TARGET;
 import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE;
 import static org.apache.flink.configuration.PipelineOptions.NAME;
+import static 
org.apache.flink.configuration.StateRecoveryOptions.SAVEPOINT_PATH;
 import static 
org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
 import static 
org.apache.flink.table.api.internal.TableResultInternal.TABLE_RESULT_OK;
 import static 
org.apache.flink.table.catalog.CatalogBaseTable.TableKind.MATERIALIZED_TABLE;
@@ -78,6 +87,12 @@ public class MaterializedTableManager {
 } else if (op instanceof AlterMaterializedTableRefreshOperation) {
 return callAlterMaterializedTableRefreshOperation(
 operationExecutor, handle, 
(AlterMaterializedTableRefreshOperation) op);
+} else if (op instanceof AlterMaterializedTableSuspendOperation) {
+return callAlterMaterializedTableSuspend(
+operationExecutor, handle, 
(AlterMaterializedTableSuspendOperation) op);
+} else if (op instanceof AlterMaterializedTableResumeOperation) {
+return callAlterMaterializedTableResume(
+operationExecutor, handle, 
(AlterMaterializedTableResumeOperation) op);
 }
 
 throw new SqlExecutionException(
@@ -115,6 +130,105 @@ public class MaterializedTableManager {
 CatalogMaterializedTable catalogMaterializedTable =
 createMaterializedTableOperation.getCatalogMaterializedTable();
 
+try {
+executeContinuousRefreshJob(
+operationExecutor,
+handle,
+catalogMaterializedTable,
+materializedTableIdentifier,
+Collections.emptyMap(),
+Optional.empty());
+} catch (Exception e) {
+// drop

(flink) branch master updated (9fe8d7bf870 -> e4972c003f6)

2024-05-12 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 9fe8d7bf870 [FLINK-35198][table] Support manual refresh materialized 
table
 add e80c2864db5 [FLINK-35197][table] Fix incomplete serialization and 
deserialization of materialized tables
 add 3b6e8db11fe [FLINK-35197][table] Support convert alter materialized 
table suspend/resume nodes to operations
 new e4972c003f6 [FLINK-35197][table] Support the execution of suspend, 
resume materialized table in continuous refresh mode

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../MaterializedTableManager.java  | 215 ++-
 .../service/MaterializedTableStatementITCase.java  | 302 -
 .../MaterializedTableManagerTest.java  |  39 +++
 ... => AlterMaterializedTableResumeOperation.java} |  43 ++-
 .../AlterMaterializedTableSuspendOperation.java}   |  23 +-
 .../catalog/CatalogBaseTableResolutionTest.java|  73 -
 .../flink/table/catalog/CatalogPropertiesUtil.java |  10 +-
 .../table/refresh/ContinuousRefreshHandler.java|  22 +-
 ... SqlAlterMaterializedTableResumeConverter.java} |  36 ++-
 ...SqlAlterMaterializedTableSuspendConverter.java} |  22 +-
 .../operations/converters/SqlNodeConverters.java   |   2 +
 ...erializedTableNodeToOperationConverterTest.java |  40 ++-
 12 files changed, 735 insertions(+), 92 deletions(-)
 copy 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/{AlterMaterializedTableRefreshOperation.java
 => AlterMaterializedTableResumeOperation.java} (56%)
 copy 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/{command/ShowJobsOperation.java
 => materializedtable/AlterMaterializedTableSuspendOperation.java} (63%)
 copy 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/{SqlAlterMaterializedTableRefreshConverter.java
 => SqlAlterMaterializedTableResumeConverter.java} (54%)
 copy 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/{SqlAlterMaterializedTableRefreshConverter.java
 => SqlAlterMaterializedTableSuspendConverter.java} (69%)



(flink) branch master updated (86c8304d735 -> 9fe8d7bf870)

2024-05-11 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 86c8304d735 [FLINK-35041][test] Fix the 
IncrementalRemoteKeyedStateHandleTest.testSharedStateReRegistration failed
 add 9fe8d7bf870 [FLINK-35198][table] Support manual refresh materialized 
table

No new revisions were added by this update.

Summary of changes:
 .../MaterializedTableManager.java  | 144 -
 .../service/MaterializedTableStatementITCase.java  | 238 +
 .../gateway/service/SqlGatewayServiceITCase.java   |  30 +--
 .../MaterializedTableManagerTest.java  |  54 +
 .../service/utils/SqlGatewayServiceTestUtil.java   |  19 ++
 .../sql/parser/ddl/SqlAlterMaterializedTable.java  |   4 +
 .../ddl/SqlAlterMaterializedTableRefresh.java  |  10 +-
 .../flink/table/operations/OperationUtils.java |   6 +-
 .../AlterMaterializedTableRefreshOperation.java|  68 ++
 ...SqlAlterMaterializedTableRefreshConverter.java} |  31 ++-
 .../operations/converters/SqlNodeConverters.java   |   1 +
 ...erializedTableNodeToOperationConverterTest.java |  30 +++
 12 files changed, 590 insertions(+), 45 deletions(-)
 create mode 100644 
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManagerTest.java
 create mode 100644 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/AlterMaterializedTableRefreshOperation.java
 copy 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/{SqlAlterTableDropPartitionConverter.java
 => SqlAlterMaterializedTableRefreshConverter.java} (53%)



(flink) branch release-1.19 updated: [FLINK-35184][table-runtime] Fix mini-batch join hash collision when use InputSideHasNoUniqueKeyBundle (#24749)

2024-05-09 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.19
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.19 by this push:
 new 17e7c3eaf14 [FLINK-35184][table-runtime] Fix mini-batch join hash 
collision when use InputSideHasNoUniqueKeyBundle (#24749)
17e7c3eaf14 is described below

commit 17e7c3eaf14b6c63f55d28a308e30ad6a3a80c95
Author: Roman Boyko 
AuthorDate: Fri May 10 10:57:45 2024 +0700

[FLINK-35184][table-runtime] Fix mini-batch join hash collision when use 
InputSideHasNoUniqueKeyBundle (#24749)
---
 .../bundle/InputSideHasNoUniqueKeyBundle.java  | 25 --
 .../join/stream/StreamingJoinOperatorTestBase.java |  4 +-
 .../stream/StreamingMiniBatchJoinOperatorTest.java | 95 +-
 3 files changed, 93 insertions(+), 31 deletions(-)

diff --git 
a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/bundle/InputSideHasNoUniqueKeyBundle.java
 
b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/bundle/InputSideHasNoUniqueKeyBundle.java
index b5738835b95..fdc9e1d5193 100644
--- 
a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/bundle/InputSideHasNoUniqueKeyBundle.java
+++ 
b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/bundle/InputSideHasNoUniqueKeyBundle.java
@@ -96,15 +96,26 @@ public class InputSideHasNoUniqueKeyBundle extends 
BufferBundle leftTypeInfo =
+protected InternalTypeInfo leftTypeInfo =
 InternalTypeInfo.of(
 RowType.of(
 new LogicalType[] {
@@ -57,7 +57,7 @@ public abstract class StreamingJoinOperatorTestBase {
 new LogicalType[] {new CharType(false, 20), new 
CharType(true, 10)},
 new String[] {"line_order_id0", 
"line_order_ship_mode"}));
 
-protected final RowDataKeySelector leftKeySelector =
+protected RowDataKeySelector leftKeySelector =
 HandwrittenSelectorUtil.getRowDataSelector(
 new int[] {1},
 leftTypeInfo.toRowType().getChildren().toArray(new 
LogicalType[0]));
diff --git 
a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/stream/StreamingMiniBatchJoinOperatorTest.java
 
b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/stream/StreamingMiniBatchJoinOperatorTest.java
index 62b8116a0b0..7e92f72cf5e 100644
--- 
a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/stream/StreamingMiniBatchJoinOperatorTest.java
+++ 
b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/stream/StreamingMiniBatchJoinOperatorTest.java
@@ -25,13 +25,13 @@ import 
org.apache.flink.table.runtime.operators.bundle.trigger.CountCoBundleTrig
 import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
 import 
org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
 import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.BigIntType;
 import org.apache.flink.table.types.logical.CharType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.utils.HandwrittenSelectorUtil;
 import org.apache.flink.types.RowKind;
 
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Tag;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.TestInfo;
@@ -55,27 +55,6 @@ public final class StreamingMiniBatchJoinOperatorTest 
extends StreamingJoinOpera
 private RowDataKeySelector leftUniqueKeySelector;
 private RowDataKeySelector rightUniqueKeySelector;
 
-@BeforeEach
-public void beforeEach(TestInfo testInfo) throws Exception {
-rightTypeInfo =
-InternalTypeInfo.of(
-RowType.of(
-new LogicalType[] {
-new CharType(false, 20),
-new CharType(false, 20),
-new CharType(true, 10)
-},
-new String[] {
-"order_id#", "line_order_id0", 
"line_order_ship_mode"
-}));
-
-rightKeySelector =
-HandwrittenSelectorUtil.getRowDataSelector(
-new int[] {1},
-rightTypeInfo.toRowType().getChildren().toArray(new 
LogicalType[0]));
-super.beforeEach(testInfo);
-}
-
 @

(flink) 01/08: [FLINK-35195][test/test-filesystem] test-filesystem Catalog support create generic table

2024-05-07 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 84f0632b15c2a192aa22a525c7b4937f80f20a34
Author: fengli 
AuthorDate: Tue Apr 30 16:23:25 2024 +0800

[FLINK-35195][test/test-filesystem] test-filesystem Catalog support create 
generic table
---
 .../file/testutils/TestFileSystemTableFactory.java | 35 +-
 .../testutils/catalog/TestFileSystemCatalog.java   | 26 +--
 .../catalog/TestFileSystemCatalogITCase.java   | 79 +-
 .../catalog/TestFileSystemCatalogTest.java | 38 +++
 4 files changed, 170 insertions(+), 8 deletions(-)

diff --git 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/TestFileSystemTableFactory.java
 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/TestFileSystemTableFactory.java
index aa5cd5e17bb..c7009af581c 100644
--- 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/TestFileSystemTableFactory.java
+++ 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/TestFileSystemTableFactory.java
@@ -22,10 +22,14 @@ import org.apache.flink.annotation.Internal;
 import org.apache.flink.connector.file.table.FileSystemTableFactory;
 import org.apache.flink.connector.file.table.TestFileSystemTableSource;
 import org.apache.flink.connector.file.table.factories.BulkReaderFormatFactory;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
 import org.apache.flink.table.connector.source.DynamicTableSource;
 import org.apache.flink.table.factories.DeserializationFormatFactory;
 import org.apache.flink.table.factories.Factory;
 import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.file.testutils.catalog.TestFileSystemCatalog;
+
+import java.util.Collections;
 
 /** Test filesystem {@link Factory}. */
 @Internal
@@ -40,9 +44,21 @@ public class TestFileSystemTableFactory extends 
FileSystemTableFactory {
 
 @Override
 public DynamicTableSource createDynamicTableSource(Context context) {
+final boolean isFileSystemTable =
+
TestFileSystemCatalog.isFileSystemTable(context.getCatalogTable().getOptions());
+if (!isFileSystemTable) {
+return FactoryUtil.createDynamicTableSource(
+null,
+context.getObjectIdentifier(),
+context.getCatalogTable(),
+Collections.emptyMap(),
+context.getConfiguration(),
+context.getClassLoader(),
+context.isTemporary());
+}
+
 FactoryUtil.TableFactoryHelper helper = 
FactoryUtil.createTableFactoryHelper(this, context);
 validate(helper);
-
 return new TestFileSystemTableSource(
 context.getObjectIdentifier(),
 context.getPhysicalRowDataType(),
@@ -51,4 +67,21 @@ public class TestFileSystemTableFactory extends 
FileSystemTableFactory {
 discoverDecodingFormat(context, BulkReaderFormatFactory.class),
 discoverDecodingFormat(context, 
DeserializationFormatFactory.class));
 }
+
+@Override
+public DynamicTableSink createDynamicTableSink(Context context) {
+final boolean isFileSystemTable =
+
TestFileSystemCatalog.isFileSystemTable(context.getCatalogTable().getOptions());
+if (!isFileSystemTable) {
+return FactoryUtil.createDynamicTableSink(
+null,
+context.getObjectIdentifier(),
+context.getCatalogTable(),
+Collections.emptyMap(),
+context.getConfiguration(),
+context.getClassLoader(),
+context.isTemporary());
+}
+return super.createDynamicTableSink(context);
+}
 }
diff --git 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalog.java
 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalog.java
index 490dd29d608..6d64ecee032 100644
--- 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalog.java
+++ 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalog.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.file.testutils.catalog;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.java.tuple.Tuple4;
 import

(flink) 06/08: [FLINK-35195][table] Introduce MaterializedTableChange to support update materialized table refresh status and RefreshHandler

2024-05-07 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 192e1e8fb04c3a8c88673fde1b66dd359b5b0fe0
Author: fengli 
AuthorDate: Mon May 6 20:21:09 2024 +0800

[FLINK-35195][table] Introduce MaterializedTableChange to support update 
materialized table refresh status and RefreshHandler
---
 .../apache/flink/table/catalog/CatalogManager.java |   3 +-
 .../operations/ddl/AlterTableChangeOperation.java  |   6 +-
 .../AlterMaterializedTableChangeOperation.java | 107 ++
 .../AlterMaterializedTableOperation.java   |  42 
 .../apache/flink/table/catalog/TableChange.java| 120 +
 5 files changed, 275 insertions(+), 3 deletions(-)

diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java
index 9e7bf5ec007..51b69c650eb 100644
--- 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java
+++ 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java
@@ -1151,7 +1151,8 @@ public final class CatalogManager implements 
CatalogRegistry, AutoCloseable {
 (catalog, path) -> {
 final CatalogBaseTable resolvedTable = 
resolveCatalogBaseTable(table);
 catalog.alterTable(path, resolvedTable, ignoreIfNotExists);
-if (resolvedTable instanceof CatalogTable) {
+if (resolvedTable instanceof CatalogTable
+|| resolvedTable instanceof 
CatalogMaterializedTable) {
 catalogModificationListeners.forEach(
 listener ->
 listener.onEvent(
diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/AlterTableChangeOperation.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/AlterTableChangeOperation.java
index 158bdd22121..7a597415235 100644
--- 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/AlterTableChangeOperation.java
+++ 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/AlterTableChangeOperation.java
@@ -58,7 +58,9 @@ public class AlterTableChangeOperation extends 
AlterTableOperation {
 @Override
 public String asSummaryString() {
 String changes =
-
tableChanges.stream().map(this::toString).collect(Collectors.joining(",\n"));
+tableChanges.stream()
+.map(AlterTableChangeOperation::toString)
+.collect(Collectors.joining(",\n"));
 return String.format(
 "ALTER TABLE %s%s\n%s",
 ignoreIfTableNotExists ? "IF EXISTS " : "",
@@ -66,7 +68,7 @@ public class AlterTableChangeOperation extends 
AlterTableOperation {
 changes);
 }
 
-private String toString(TableChange tableChange) {
+public static String toString(TableChange tableChange) {
 if (tableChange instanceof TableChange.SetOption) {
 TableChange.SetOption setChange = (TableChange.SetOption) 
tableChange;
 return String.format("  SET '%s' = '%s'", setChange.getKey(), 
setChange.getValue());
diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/AlterMaterializedTableChangeOperation.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/AlterMaterializedTableChangeOperation.java
new file mode 100644
index 000..49f220a8ddc
--- /dev/null
+++ 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/AlterMaterializedTableChangeOperation.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * lim

(flink) 07/08: [FLINK-35195][table] Introduce DropMaterializedTableOperation to support drop materialized table

2024-05-07 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit e0d342a2a6ba320c1fe0f7a42239254d25f95fd5
Author: fengli 
AuthorDate: Mon May 6 20:22:19 2024 +0800

[FLINK-35195][table] Introduce DropMaterializedTableOperation to support 
drop materialized table
---
 .../DropMaterializedTableOperation.java| 54 ++
 1 file changed, 54 insertions(+)

diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/DropMaterializedTableOperation.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/DropMaterializedTableOperation.java
new file mode 100644
index 000..e5eee557bfc
--- /dev/null
+++ 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/DropMaterializedTableOperation.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.table.operations.materializedtable;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.operations.Operation;
+import org.apache.flink.table.operations.OperationUtils;
+import org.apache.flink.table.operations.ddl.DropTableOperation;
+
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/** Operation to describe a DROP MATERIALIZED TABLE statement. */
+@Internal
+public class DropMaterializedTableOperation extends DropTableOperation
+implements MaterializedTableOperation {
+
+public DropMaterializedTableOperation(
+ObjectIdentifier tableIdentifier, boolean ifExists, boolean 
isTemporary) {
+super(tableIdentifier, ifExists, isTemporary);
+}
+
+@Override
+public String asSummaryString() {
+Map params = new LinkedHashMap<>();
+params.put("identifier", getTableIdentifier());
+params.put("IfExists", isIfExists());
+params.put("isTemporary", isTemporary());
+
+return OperationUtils.formatWithChildren(
+"DROP MATERIALIZED TABLE",
+params,
+Collections.emptyList(),
+Operation::asSummaryString);
+}
+}



(flink) 05/08: [FLINK-35195][table] Introduce ContinuousRefreshHandler and serializer for continuous refresh mode

2024-05-07 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit c99eb54ce8f362d970c173a2a579e8fc28ac
Author: fengli 
AuthorDate: Mon May 6 20:19:49 2024 +0800

[FLINK-35195][table] Introduce ContinuousRefreshHandler and serializer for 
continuous refresh mode
---
 .../table/refresh/ContinuousRefreshHandler.java| 50 ++
 .../ContinuousRefreshHandlerSerializer.java| 44 +++
 2 files changed, 94 insertions(+)

diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/refresh/ContinuousRefreshHandler.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/refresh/ContinuousRefreshHandler.java
new file mode 100644
index 000..60a92bed02e
--- /dev/null
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/refresh/ContinuousRefreshHandler.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.table.refresh;
+
+import org.apache.flink.annotation.Internal;
+
+import java.io.Serializable;
+
+/** Embedded continuous refresh handler of Flink streaming job for 
materialized table. */
+@Internal
+public class ContinuousRefreshHandler implements RefreshHandler, Serializable {
+
+// TODO: add clusterId for yarn and k8s resource manager
+private final String executionTarget;
+private final String jobId;
+
+public ContinuousRefreshHandler(String executionTarget, String jobId) {
+this.executionTarget = executionTarget;
+this.jobId = jobId;
+}
+
+public String getExecutionTarget() {
+return executionTarget;
+}
+
+public String getJobId() {
+return jobId;
+}
+
+@Override
+public String asSummaryString() {
+return String.format("{\n executionTarget: %s,\n jobId: %s\n}", 
executionTarget, jobId);
+}
+}
diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/refresh/ContinuousRefreshHandlerSerializer.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/refresh/ContinuousRefreshHandlerSerializer.java
new file mode 100644
index 000..f62ccc99e09
--- /dev/null
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/refresh/ContinuousRefreshHandlerSerializer.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.refresh;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.InstantiationUtil;
+
+import java.io.IOException;
+
+/** Serializer for {@link ContinuousRefreshHandler}. */
+@Internal
+public class ContinuousRefreshHandlerSerializer
+implements RefreshHandlerSerializer {
+
+public static final ContinuousRefreshHandlerSerializer INSTANCE =
+new ContinuousRefreshHandlerSerializer();
+
+@Override
+public byte[] serialize(ContinuousRefreshHandler refreshHandler) throws 
IOException {
+return InstantiationUtil.serializeObject(refreshHandler);
+}
+
+@Override
+public ContinuousRefreshHandler deserialize(byte[] serializedBytes, 
ClassLoader cl)
+throws IOException, ClassNotFoundException {
+return InstantiationUtil.deserializeObject(serializedBytes, cl);
+}
+}



(flink) 08/08: [FLINK-35195][table] Support execute CreateMaterializedTableOperation for continuous refresh mode in SqlGateway

2024-05-07 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 29736b8c01924b7da03d4bcbfd9c812a8e5a08b4
Author: fengli 
AuthorDate: Mon May 6 20:24:16 2024 +0800

[FLINK-35195][table] Support execute CreateMaterializedTableOperation for 
continuous refresh mode in SqlGateway
---
 flink-table/flink-sql-gateway/pom.xml  |   6 +
 .../MaterializedTableManager.java  | 182 ++
 .../service/operation/OperationExecutor.java   |  25 +-
 .../service/MaterializedTableStatementITCase.java  | 274 +
 4 files changed, 483 insertions(+), 4 deletions(-)

diff --git a/flink-table/flink-sql-gateway/pom.xml 
b/flink-table/flink-sql-gateway/pom.xml
index 1a50d665a18..61f1e75942e 100644
--- a/flink-table/flink-sql-gateway/pom.xml
+++ b/flink-table/flink-sql-gateway/pom.xml
@@ -127,6 +127,12 @@
test-jar
test

+
+org.apache.flink
+flink-table-filesystem-test-utils
+${project.version}
+test
+
 
 
 
diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
new file mode 100644
index 000..fed60634a3a
--- /dev/null
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.table.gateway.service.materializedtable;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.catalog.CatalogMaterializedTable;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.TableChange;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.gateway.api.operation.OperationHandle;
+import org.apache.flink.table.gateway.api.results.ResultSet;
+import org.apache.flink.table.gateway.service.operation.OperationExecutor;
+import org.apache.flink.table.gateway.service.result.ResultFetcher;
+import org.apache.flink.table.gateway.service.utils.SqlExecutionException;
+import 
org.apache.flink.table.operations.materializedtable.AlterMaterializedTableChangeOperation;
+import 
org.apache.flink.table.operations.materializedtable.CreateMaterializedTableOperation;
+import 
org.apache.flink.table.operations.materializedtable.DropMaterializedTableOperation;
+import 
org.apache.flink.table.operations.materializedtable.MaterializedTableOperation;
+import org.apache.flink.table.refresh.ContinuousRefreshHandler;
+import org.apache.flink.table.refresh.ContinuousRefreshHandlerSerializer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.api.common.RuntimeExecutionMode.STREAMING;
+import static org.apache.flink.configuration.DeploymentOptions.TARGET;
+import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE;
+import static org.apache.flink.configuration.PipelineOptions.NAME;
+import static 
org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static 
org.apache.flink.table.api.internal.TableResultInternal.TABLE_RESULT_OK;
+
+/** Manager is responsible for execute the {@link MaterializedTableOperation}. 
*/
+@Internal
+public class MaterializedTableManager {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(MaterializedTableManager.class);
+
+public static ResultFetcher callMaterializedTableOperation(
+OperationExecutor operationExecutor,
+OperationHandle handle,
+MaterializedTableOperation op,
+String statement) {
+if (op instanceof CreateMateri

(flink) 04/08: [FLINK-35195][table] Convert SqlCreateMaterializedTable node to CreateMaterializedTableOperation

2024-05-07 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit e28e495cdd3e0e7cbb58685cb09e1fa08af7223e
Author: fengli 
AuthorDate: Mon May 6 20:17:57 2024 +0800

[FLINK-35195][table] Convert SqlCreateMaterializedTable node to 
CreateMaterializedTableOperation
---
 .../flink/sql/parser/SqlConstraintValidator.java   |   2 +-
 .../sql/parser/ddl/SqlCreateMaterializedTable.java |   1 -
 .../CreateMaterializedTableOperation.java  |  76 ++
 .../MaterializedTableOperation.java|  26 +++
 .../planner/operations/SqlNodeConvertContext.java  |   8 +
 .../SqlCreateMaterializedTableConverter.java   | 210 +
 .../operations/converters/SqlNodeConverter.java|   5 +
 .../operations/converters/SqlNodeConverters.java   |   1 +
 .../planner/utils/MaterializedTableUtils.java  |  98 
 ...erializedTableNodeToOperationConverterTest.java | 259 +
 .../SqlNodeToOperationConversionTestBase.java  |   2 +-
 .../SqlRTASNodeToOperationConverterTest.java   |   2 +-
 12 files changed, 686 insertions(+), 4 deletions(-)

diff --git 
a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/SqlConstraintValidator.java
 
b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/SqlConstraintValidator.java
index 8a9a7727b54..f157e5034a8 100644
--- 
a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/SqlConstraintValidator.java
+++ 
b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/SqlConstraintValidator.java
@@ -89,7 +89,7 @@ public class SqlConstraintValidator {
 }
 
 /** Check table constraint. */
-private static void validate(SqlTableConstraint constraint) throws 
SqlValidateException {
+public static void validate(SqlTableConstraint constraint) throws 
SqlValidateException {
 if (constraint.isUnique()) {
 throw new SqlValidateException(
 constraint.getParserPosition(), "UNIQUE constraint is not 
supported yet");
diff --git 
a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateMaterializedTable.java
 
b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateMaterializedTable.java
index 1630a0f0117..eae6f1fcba9 100644
--- 
a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateMaterializedTable.java
+++ 
b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateMaterializedTable.java
@@ -132,7 +132,6 @@ public class SqlCreateMaterializedTable extends SqlCreate {
 return freshness;
 }
 
-@Nullable
 public Optional getRefreshMode() {
 return Optional.ofNullable(refreshMode);
 }
diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/CreateMaterializedTableOperation.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/CreateMaterializedTableOperation.java
new file mode 100644
index 000..d4eff00254d
--- /dev/null
+++ 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/CreateMaterializedTableOperation.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.operations.materializedtable;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.internal.TableResultImpl;
+import org.apache.flink.table.api.internal.TableResultInternal;
+import org.apache.flink.table.catalog.CatalogMaterializedTable;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable;
+import org.apache.flink.table.operations.Operation;
+import org.apache.flink.table.operations.OperationUtils;
+import org.apache.flink.table.operations.ddl.CreateOperation;
+
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/** Operation to describe a CREATE MATERIALIZED TABLE st

(flink) 03/08: [FLINK-35195][table] Convert CatalogMaterializedTable to CatalogTable to generate execution plan for planner

2024-05-07 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit d8491c0f9c07f0d3d5e1428cad54902acb6ae0d0
Author: fengli 
AuthorDate: Mon May 6 20:12:54 2024 +0800

[FLINK-35195][table] Convert CatalogMaterializedTable to CatalogTable to 
generate execution plan for planner
---
 .../flink/table/catalog/ContextResolvedTable.java  | 26 ++
 .../catalog/ResolvedCatalogMaterializedTable.java  | 13 +++
 .../planner/catalog/DatabaseCalciteSchema.java |  3 ++-
 .../operations/SqlNodeToOperationConversion.java   |  4 +++-
 4 files changed, 44 insertions(+), 2 deletions(-)

diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/ContextResolvedTable.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/ContextResolvedTable.java
index e7b9e5f0835..70a0b5c16d0 100644
--- 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/ContextResolvedTable.java
+++ 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/ContextResolvedTable.java
@@ -142,6 +142,20 @@ public final class ContextResolvedTable {
 return (T) resolvedTable.getOrigin();
 }
 
+/**
+ * Convert the {@link ResolvedCatalogMaterializedTable} in {@link 
ContextResolvedTable} to
+ * {@link ResolvedCatalogTable }.
+ */
+public ContextResolvedTable toCatalogTable() {
+if (resolvedTable.getTableKind() == 
CatalogBaseTable.TableKind.MATERIALIZED_TABLE) {
+return ContextResolvedTable.permanent(
+objectIdentifier,
+catalog,
+((ResolvedCatalogMaterializedTable) 
resolvedTable).toResolvedCatalogTable());
+}
+return this;
+}
+
 /**
  * Copy the {@link ContextResolvedTable}, replacing the underlying {@link 
CatalogTable} options.
  */
@@ -150,6 +164,12 @@ public final class ContextResolvedTable {
 throw new ValidationException(
 String.format("View '%s' cannot be enriched with new 
options.", this));
 }
+if (resolvedTable.getTableKind() == 
CatalogBaseTable.TableKind.MATERIALIZED_TABLE) {
+return ContextResolvedTable.permanent(
+objectIdentifier,
+catalog,
+((ResolvedCatalogMaterializedTable) 
resolvedTable).copy(newOptions));
+}
 return new ContextResolvedTable(
 objectIdentifier,
 catalog,
@@ -159,6 +179,12 @@ public final class ContextResolvedTable {
 
 /** Copy the {@link ContextResolvedTable}, replacing the underlying {@link 
ResolvedSchema}. */
 public ContextResolvedTable copy(ResolvedSchema newSchema) {
+if (resolvedTable.getTableKind() == 
CatalogBaseTable.TableKind.MATERIALIZED_TABLE) {
+throw new ValidationException(
+String.format(
+"Materialized table '%s' cannot be copied with new 
schema %s.",
+this, newSchema));
+}
 return new ContextResolvedTable(
 objectIdentifier,
 catalog,
diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/ResolvedCatalogMaterializedTable.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/ResolvedCatalogMaterializedTable.java
index a0206af3111..f876cd74c4d 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/ResolvedCatalogMaterializedTable.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/ResolvedCatalogMaterializedTable.java
@@ -182,4 +182,17 @@ public class ResolvedCatalogMaterializedTable
 + resolvedSchema
 + '}';
 }
+
+/** Convert this object to a {@link ResolvedCatalogTable} object for 
planner optimize query. */
+public ResolvedCatalogTable toResolvedCatalogTable() {
+return new ResolvedCatalogTable(
+CatalogTable.newBuilder()
+.schema(getUnresolvedSchema())
+.comment(getComment())
+.partitionKeys(getPartitionKeys())
+.options(getOptions())
+.snapshot(getSnapshot().orElse(null))
+.build(),
+getResolvedSchema());
+}
 }
diff --git 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java
 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java
index 7ba1e04d83e..d3e738ae5ff 100644
--- 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/catalog/Databas

(flink) 02/08: [FLINK-35195][table] Introduce materialized table reflated config options

2024-05-07 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit b037f56b10c7dce4505ed6f4e28030350742a523
Author: fengli 
AuthorDate: Mon May 6 20:07:09 2024 +0800

[FLINK-35195][table] Introduce materialized table reflated config options
---
 docs/content.zh/docs/dev/table/config.md   |  6 +++
 docs/content/docs/dev/table/config.md  |  6 +++
 .../materialized_table_config_configuration.html   | 24 +
 .../api/config/MaterializedTableConfigOptions.java | 59 ++
 4 files changed, 95 insertions(+)

diff --git a/docs/content.zh/docs/dev/table/config.md 
b/docs/content.zh/docs/dev/table/config.md
index f1a0be083c5..1748fde1b72 100644
--- a/docs/content.zh/docs/dev/table/config.md
+++ b/docs/content.zh/docs/dev/table/config.md
@@ -134,6 +134,12 @@ Flink SQL> SET 'table.exec.mini-batch.size' = '5000';
 
 {{< generated/table_config_configuration >}}
 
+### Materialized Table 配置
+
+以下配置可以用于调整 Materialized Table 的行为。
+
+{{< generated/materialized_table_config_configuration >}}
+
 ### SQL Client 配置
 
 以下配置可以用于调整 sql client 的行为。
diff --git a/docs/content/docs/dev/table/config.md 
b/docs/content/docs/dev/table/config.md
index 697d820db4c..51a264414c8 100644
--- a/docs/content/docs/dev/table/config.md
+++ b/docs/content/docs/dev/table/config.md
@@ -149,6 +149,12 @@ The following options can be used to adjust the behavior 
of the table planner.
 
 {{< generated/table_config_configuration >}}
 
+### Materialized Table Options
+
+The following options can be used to adjust the behavior of the materialized 
table.
+
+{{< generated/materialized_table_config_configuration >}}
+
 ### SQL Client Options
 
 The following options can be used to adjust the behavior of the sql client.
diff --git 
a/docs/layouts/shortcodes/generated/materialized_table_config_configuration.html
 
b/docs/layouts/shortcodes/generated/materialized_table_config_configuration.html
new file mode 100644
index 000..d5829bf3224
--- /dev/null
+++ 
b/docs/layouts/shortcodes/generated/materialized_table_config_configuration.html
@@ -0,0 +1,24 @@
+
+
+
+Key
+Default
+Type
+Description
+
+
+
+
+
materialized-table.refresh-mode.freshness-threshold Batch Streaming
+30 min
+Duration
+Specifies a time threshold for determining the materialized 
table refresh mode. If the materialized table defined FRESHNESS is below this 
threshold, it run in continuous mode. Otherwise, it switches to full refresh 
mode.
+
+
+partition.fields.#.date-formatter Batch Streaming
+(none)
+String
+Specifies the time partition formatter for the partitioned 
materialized table, where '#' denotes a string-based partition field name. This 
serves as a hint to the framework regarding which partition to refresh in full 
refresh mode.
+
+
+
diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/MaterializedTableConfigOptions.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/MaterializedTableConfigOptions.java
new file mode 100644
index 000..b08466e05ab
--- /dev/null
+++ 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/MaterializedTableConfigOptions.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.table.api.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigOption;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * This class holds {@link org.apache.flink.configuration.ConfigOption}s used 
by table module for
+ * materialized table.
+ */
+@PublicEvolving
+public class MaterializedTableConfigOptions {
+
+private MaterializedTableConfigOptions() {}
+
+

(flink) branch master updated (ea4112aefa7 -> 29736b8c019)

2024-05-07 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from ea4112aefa7 [FLINK-35161][state] Implement StateExecutor for 
ForStStateBackend
 new 84f0632b15c [FLINK-35195][test/test-filesystem] test-filesystem 
Catalog support create generic table
 new b037f56b10c [FLINK-35195][table] Introduce materialized table reflated 
config options
 new d8491c0f9c0 [FLINK-35195][table] Convert CatalogMaterializedTable to 
CatalogTable to generate execution plan for planner
 new e28e495cdd3 [FLINK-35195][table] Convert SqlCreateMaterializedTable 
node to CreateMaterializedTableOperation
 new c99eb54ce8f [FLINK-35195][table] Introduce ContinuousRefreshHandler 
and serializer for continuous refresh mode
 new 192e1e8fb04 [FLINK-35195][table] Introduce MaterializedTableChange to 
support update materialized table refresh status and RefreshHandler
 new e0d342a2a6b [FLINK-35195][table] Introduce 
DropMaterializedTableOperation to support drop materialized table
 new 29736b8c019 [FLINK-35195][table] Support execute 
CreateMaterializedTableOperation for continuous refresh mode in SqlGateway

The 8 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 docs/content.zh/docs/dev/table/config.md   |   6 +
 docs/content/docs/dev/table/config.md  |   6 +
 .../materialized_table_config_configuration.html   |  24 ++
 flink-table/flink-sql-gateway/pom.xml  |   6 +
 .../MaterializedTableManager.java  | 182 ++
 .../service/operation/OperationExecutor.java   |  25 +-
 .../service/MaterializedTableStatementITCase.java  | 274 +
 .../flink/sql/parser/SqlConstraintValidator.java   |   2 +-
 .../sql/parser/ddl/SqlCreateMaterializedTable.java |   1 -
 .../api/config/MaterializedTableConfigOptions.java |  59 +
 .../apache/flink/table/catalog/CatalogManager.java |   3 +-
 .../flink/table/catalog/ContextResolvedTable.java  |  26 ++
 .../operations/ddl/AlterTableChangeOperation.java  |   6 +-
 .../AlterMaterializedTableChangeOperation.java | 107 
 .../AlterMaterializedTableOperation.java   |  42 
 .../CreateMaterializedTableOperation.java  |  76 ++
 .../DropMaterializedTableOperation.java|  54 
 .../MaterializedTableOperation.java|  26 ++
 .../catalog/ResolvedCatalogMaterializedTable.java  |  13 +
 .../apache/flink/table/catalog/TableChange.java| 120 +
 .../table/refresh/ContinuousRefreshHandler.java|  50 
 .../ContinuousRefreshHandlerSerializer.java|  44 
 .../planner/catalog/DatabaseCalciteSchema.java |   3 +-
 .../planner/operations/SqlNodeConvertContext.java  |   8 +
 .../operations/SqlNodeToOperationConversion.java   |   4 +-
 .../SqlCreateMaterializedTableConverter.java   | 210 
 .../operations/converters/SqlNodeConverter.java|   5 +
 .../operations/converters/SqlNodeConverters.java   |   1 +
 .../planner/utils/MaterializedTableUtils.java  |  98 
 ...erializedTableNodeToOperationConverterTest.java | 259 +++
 .../SqlNodeToOperationConversionTestBase.java  |   2 +-
 .../SqlRTASNodeToOperationConverterTest.java   |   2 +-
 .../file/testutils/TestFileSystemTableFactory.java |  35 ++-
 .../testutils/catalog/TestFileSystemCatalog.java   |  26 +-
 .../catalog/TestFileSystemCatalogITCase.java   |  79 +-
 .../catalog/TestFileSystemCatalogTest.java |  38 +++
 36 files changed, 1901 insertions(+), 21 deletions(-)
 create mode 100644 
docs/layouts/shortcodes/generated/materialized_table_config_configuration.html
 create mode 100644 
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/materializedtable/MaterializedTableManager.java
 create mode 100644 
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/MaterializedTableStatementITCase.java
 create mode 100644 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/MaterializedTableConfigOptions.java
 create mode 100644 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/AlterMaterializedTableChangeOperation.java
 create mode 100644 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/AlterMaterializedTableOperation.java
 create mode 100644 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/materializedtable/CreateMaterializedTableOperation.java
 create mode 100644 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operatio

(flink) branch release-1.17 updated: [FLINK-34379][table] Fix OutOfMemoryError with large queries

2024-05-05 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch release-1.17
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.17 by this push:
 new d2f93a5527b [FLINK-34379][table] Fix OutOfMemoryError with large 
queries
d2f93a5527b is described below

commit d2f93a5527b05583fc97bbae511ca0ac95325c02
Author: Jeyhun Karimov 
AuthorDate: Tue Apr 2 00:24:02 2024 +0200

[FLINK-34379][table] Fix OutOfMemoryError with large queries
---
 .../utils/DynamicPartitionPruningUtils.java|   9 +-
 .../DynamicPartitionPruningProgramTest.java|  85 +++
 .../program/DynamicPartitionPruningProgramTest.xml | 618 +
 3 files changed, 711 insertions(+), 1 deletion(-)

diff --git 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DynamicPartitionPruningUtils.java
 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DynamicPartitionPruningUtils.java
index 90f7b40bc0b..089e1fd 100644
--- 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DynamicPartitionPruningUtils.java
+++ 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DynamicPartitionPruningUtils.java
@@ -61,8 +61,10 @@ import org.apache.calcite.util.ImmutableIntList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Optional;
+import java.util.Set;
 import java.util.stream.Collectors;
 
 /** Planner utils for Dynamic partition Pruning. */
@@ -115,7 +117,7 @@ public class DynamicPartitionPruningUtils {
 private final RelNode relNode;
 private boolean hasFilter;
 private boolean hasPartitionedScan;
-private final List tables = new ArrayList<>();
+private final Set tables = new HashSet<>();
 
 public DppDimSideChecker(RelNode relNode) {
 this.relNode = relNode;
@@ -235,9 +237,14 @@ public class DynamicPartitionPruningUtils {
 if (tables.size() == 0) {
 tables.add(catalogTable);
 } else {
+boolean hasAdded = false;
 for (ContextResolvedTable thisTable : new ArrayList<>(tables)) 
{
+if (hasAdded) {
+break;
+}
 if 
(!thisTable.getIdentifier().equals(catalogTable.getIdentifier())) {
 tables.add(catalogTable);
+hasAdded = true;
 }
 }
 }
diff --git 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/DynamicPartitionPruningProgramTest.java
 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/DynamicPartitionPruningProgramTest.java
index 8e957e2958a..c7ab3e40ef8 100644
--- 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/DynamicPartitionPruningProgramTest.java
+++ 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/DynamicPartitionPruningProgramTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.planner.plan.optimize.program;
 
+import org.apache.flink.table.api.Table;
 import org.apache.flink.table.api.TableConfig;
 import org.apache.flink.table.api.config.OptimizerConfigOptions;
 import org.apache.flink.table.catalog.ObjectPath;
@@ -30,6 +31,11 @@ import org.apache.flink.table.planner.utils.TableTestBase;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.table.api.Expressions.col;
+
 /**
  * Tests for rules that extend {@link FlinkDynamicPartitionPruningProgram} to 
create {@link
  * 
org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalDynamicFilteringTableSourceScan}.
@@ -80,6 +86,85 @@ public class DynamicPartitionPruningProgramTest extends 
TableTestBase {
 + ")");
 }
 
+@Test
+public void testLargeQueryPlanShouldNotOutOfMemoryWithTableApi() {
+// TABLE_OPTIMIZER_DYNAMIC_FILTERING_ENABLED is already enabled
+List selectStmts = new ArrayList<>();
+for (int i = 0; i < 100; i++) {
+util.tableEnv()
+.executeSql(
+"CREATE TABLE IF NOT EXISTS table"
++ i
++ "(att STRING,filename STRING) "
++ "with("
++ " 'connector' = 'values', "
++ "

(flink) branch master updated: [FLINK-35194][table] Support describe job statement for SqlGateway

2024-04-29 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 44528e0ee9f [FLINK-35194][table] Support describe job statement for 
SqlGateway
44528e0ee9f is described below

commit 44528e0ee9fbed11b5417253534078d60fed3a12
Author: xuyang 
AuthorDate: Fri Apr 26 20:29:56 2024 +0800

[FLINK-35194][table] Support describe job statement for SqlGateway

This closes #24728
---
 .../service/operation/OperationExecutor.java   | 53 +
 .../gateway/service/SqlGatewayServiceITCase.java   | 51 +
 .../src/main/codegen/data/Parser.tdd   |  5 +-
 .../src/main/codegen/includes/parserImpls.ftl  | 18 ++
 .../flink/sql/parser/dql/SqlDescribeJob.java   | 66 ++
 .../flink/sql/parser/FlinkSqlParserImplTest.java   |  6 ++
 .../operations/command/DescribeJobOperation.java   | 52 +
 .../converters/SqlDescribeJobConverter.java| 32 +++
 .../operations/converters/SqlNodeConverters.java   |  1 +
 .../table/planner/calcite/FlinkPlannerImpl.scala   |  3 +-
 10 files changed, 285 insertions(+), 2 deletions(-)

diff --git 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java
 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java
index 945265089c3..c50ba8c2bbf 100644
--- 
a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java
+++ 
b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java
@@ -83,6 +83,7 @@ import 
org.apache.flink.table.operations.StatementSetOperation;
 import org.apache.flink.table.operations.UnloadModuleOperation;
 import org.apache.flink.table.operations.UseOperation;
 import org.apache.flink.table.operations.command.AddJarOperation;
+import org.apache.flink.table.operations.command.DescribeJobOperation;
 import org.apache.flink.table.operations.command.ExecutePlanOperation;
 import org.apache.flink.table.operations.command.RemoveJarOperation;
 import org.apache.flink.table.operations.command.ResetOperation;
@@ -481,6 +482,8 @@ public class OperationExecutor {
 return callStopJobOperation(tableEnv, handle, (StopJobOperation) 
op);
 } else if (op instanceof ShowJobsOperation) {
 return callShowJobsOperation(tableEnv, handle, (ShowJobsOperation) 
op);
+} else if (op instanceof DescribeJobOperation) {
+return callDescribeJobOperation(tableEnv, handle, 
(DescribeJobOperation) op);
 } else if (op instanceof RemoveJarOperation) {
 return callRemoveJar(handle, ((RemoveJarOperation) op).getPath());
 } else if (op instanceof AddJarOperation
@@ -774,6 +777,56 @@ public class OperationExecutor {
 resultRows);
 }
 
+public ResultFetcher callDescribeJobOperation(
+TableEnvironmentInternal tableEnv,
+OperationHandle operationHandle,
+DescribeJobOperation describeJobOperation)
+throws SqlExecutionException {
+Configuration configuration = tableEnv.getConfig().getConfiguration();
+Duration clientTimeout = 
configuration.get(ClientOptions.CLIENT_TIMEOUT);
+String jobId = describeJobOperation.getJobId();
+Optional jobStatusOp =
+runClusterAction(
+configuration,
+operationHandle,
+clusterClient -> {
+try {
+JobID expectedJobId = 
JobID.fromHexString(jobId);
+return clusterClient.listJobs()
+.get(clientTimeout.toMillis(), 
TimeUnit.MILLISECONDS)
+.stream()
+.filter(job -> 
expectedJobId.equals(job.getJobId()))
+.findFirst();
+} catch (Exception e) {
+throw new SqlExecutionException(
+String.format(
+"Failed to get job %s in the 
cluster.", jobId),
+e);
+}
+});
+
+if (!jobStatusOp.isPresent()) {
+throw new SqlExecutionException(
+String.format("Described job %s does not exist in the 
cluster.", jobId));
+}
+JobStatusMessage job = jobStatusOp.get();
+
+RowData resultRow =
+GenericRowData.of(
+S

(flink) branch master updated: [FLINK-35184][table-runtime] Fix mini-batch join hash collision when use InputSideHasNoUniqueKeyBundle

2024-04-29 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new f543cc543e9 [FLINK-35184][table-runtime] Fix mini-batch join hash 
collision when use InputSideHasNoUniqueKeyBundle
f543cc543e9 is described below

commit f543cc543e9b0eb05415095190e86d3b22cdf1a4
Author: Roman Boyko 
AuthorDate: Tue Apr 23 12:13:58 2024 +0700

[FLINK-35184][table-runtime] Fix mini-batch join hash collision when use 
InputSideHasNoUniqueKeyBundle

This closes #24703
---
 .../bundle/InputSideHasNoUniqueKeyBundle.java  | 25 --
 .../join/stream/StreamingJoinOperatorTestBase.java |  4 +-
 .../stream/StreamingMiniBatchJoinOperatorTest.java | 95 +-
 3 files changed, 93 insertions(+), 31 deletions(-)

diff --git 
a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/bundle/InputSideHasNoUniqueKeyBundle.java
 
b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/bundle/InputSideHasNoUniqueKeyBundle.java
index b5738835b95..fdc9e1d5193 100644
--- 
a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/bundle/InputSideHasNoUniqueKeyBundle.java
+++ 
b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/bundle/InputSideHasNoUniqueKeyBundle.java
@@ -96,15 +96,26 @@ public class InputSideHasNoUniqueKeyBundle extends 
BufferBundle leftTypeInfo =
+protected InternalTypeInfo leftTypeInfo =
 InternalTypeInfo.of(
 RowType.of(
 new LogicalType[] {
@@ -57,7 +57,7 @@ public abstract class StreamingJoinOperatorTestBase {
 new LogicalType[] {new CharType(false, 20), new 
CharType(true, 10)},
 new String[] {"line_order_id0", 
"line_order_ship_mode"}));
 
-protected final RowDataKeySelector leftKeySelector =
+protected RowDataKeySelector leftKeySelector =
 HandwrittenSelectorUtil.getRowDataSelector(
 new int[] {1},
 leftTypeInfo.toRowType().getChildren().toArray(new 
LogicalType[0]));
diff --git 
a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/stream/StreamingMiniBatchJoinOperatorTest.java
 
b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/stream/StreamingMiniBatchJoinOperatorTest.java
index 62b8116a0b0..7e92f72cf5e 100644
--- 
a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/stream/StreamingMiniBatchJoinOperatorTest.java
+++ 
b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/stream/StreamingMiniBatchJoinOperatorTest.java
@@ -25,13 +25,13 @@ import 
org.apache.flink.table.runtime.operators.bundle.trigger.CountCoBundleTrig
 import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
 import 
org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
 import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.BigIntType;
 import org.apache.flink.table.types.logical.CharType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.utils.HandwrittenSelectorUtil;
 import org.apache.flink.types.RowKind;
 
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Tag;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.TestInfo;
@@ -55,27 +55,6 @@ public final class StreamingMiniBatchJoinOperatorTest 
extends StreamingJoinOpera
 private RowDataKeySelector leftUniqueKeySelector;
 private RowDataKeySelector rightUniqueKeySelector;
 
-@BeforeEach
-public void beforeEach(TestInfo testInfo) throws Exception {
-rightTypeInfo =
-InternalTypeInfo.of(
-RowType.of(
-new LogicalType[] {
-new CharType(false, 20),
-new CharType(false, 20),
-new CharType(true, 10)
-},
-new String[] {
-"order_id#", "line_order_id0", 
"line_order_ship_mode"
-}));
-
-rightKeySelector =
-HandwrittenSelectorUtil.getRowDataSelector(
-new int[] {1},
-rightTypeInfo.toRowType().getChildren().toArray(new 
LogicalType[0]));
-super.beforeEach(testInfo);
-}
-
 @

(flink) branch master updated: [FLINK-35191][table-api] Support alter materialized table related syntax: suspend, resume, refresh, set and reset

2024-04-29 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 330f524d185 [FLINK-35191][table-api] Support alter materialized table 
related syntax: suspend, resume, refresh, set and reset
330f524d185 is described below

commit 330f524d185d575ceb679a6c587e9c39612e844c
Author: Feng Jin 
AuthorDate: Mon Apr 29 10:21:12 2024 +0800

[FLINK-35191][table-api] Support alter materialized table related syntax: 
suspend, resume, refresh, set and reset

This closes #24737
---
 .../src/main/codegen/data/Parser.tdd   |  12 ++
 .../src/main/codegen/includes/parserImpls.ftl  |  94 +++
 .../sql/parser/ddl/SqlAlterMaterializedTable.java  |  61 +++
 .../ddl/SqlAlterMaterializedTableFreshness.java|  60 +++
 .../ddl/SqlAlterMaterializedTableOptions.java  |  67 
 .../ddl/SqlAlterMaterializedTableRefresh.java  |  61 +++
 .../ddl/SqlAlterMaterializedTableRefreshMode.java  |  62 +++
 .../parser/ddl/SqlAlterMaterializedTableReset.java |  67 
 .../ddl/SqlAlterMaterializedTableResume.java   |  74 
 .../ddl/SqlAlterMaterializedTableSuspend.java  |  47 ++
 .../flink/sql/parser/utils/ParserResource.java |   2 +-
 .../MaterializedTableStatementParserTest.java  | 187 -
 12 files changed, 792 insertions(+), 2 deletions(-)

diff --git a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd 
b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd
index dfb43353a4b..100e9edd2fb 100644
--- a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd
+++ b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd
@@ -35,6 +35,14 @@
 
"org.apache.flink.sql.parser.ddl.SqlAddPartitions.AlterTableAddPartitionContext"
 "org.apache.flink.sql.parser.ddl.SqlAlterDatabase"
 "org.apache.flink.sql.parser.ddl.SqlAlterFunction"
+"org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTable"
+"org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableFreshness"
+"org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableOptions"
+"org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableRefreshMode"
+"org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableReset"
+"org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableRefresh"
+"org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableResume"
+"org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableSuspend"
 "org.apache.flink.sql.parser.ddl.SqlAlterTable"
 "org.apache.flink.sql.parser.ddl.SqlAlterTable.AlterTableContext"
 "org.apache.flink.sql.parser.ddl.SqlAlterTableAdd"
@@ -191,6 +199,9 @@
 "STATISTICS"
 "STOP"
 "STRING"
+"SUSPEND"
+"REFRESH"
+"RESUME"
 "TABLES"
 "TIMESTAMP_LTZ"
 "TRY_CAST"
@@ -581,6 +592,7 @@
 "SqlShowCreate()"
 "SqlReplaceTable()"
 "SqlRichDescribeTable()"
+"SqlAlterMaterializedTable()"
 "SqlAlterTable()"
 "SqlAlterView()"
 "SqlShowModules()"
diff --git 
a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl 
b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl
index b52f41aa951..95509e7b8da 100644
--- a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl
+++ b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl
@@ -1779,6 +1779,100 @@ SqlCreate SqlCreateMaterializedTable(Span s, boolean 
replace, boolean isTemporar
 }
 }
 
+/**
+* Parses alter materialized table.
+*/
+SqlAlterMaterializedTable SqlAlterMaterializedTable() :
+{
+SqlParserPos startPos;
+SqlIdentifier tableIdentifier;
+SqlNodeList propertyList = SqlNodeList.EMPTY;
+SqlNodeList propertyKeyList = SqlNodeList.EMPTY;
+SqlNodeList partSpec = SqlNodeList.EMPTY;
+SqlNode freshness = null;
+}
+{
+   { startPos = getPos();}
+tableIdentifier = CompoundIdentifier()
+(
+
+{
+return new SqlAlterMaterializedTableSuspend(startPos, 
tableIdentifier);
+}
+|
+
+[  propertyList = TableProperties() ]
+{
+return new SqlAlterMaterializedTableResume(
+startPos,
+tableIdentifier,
+propertyList);
+}
+|
+
+[  {
+partSpec = new SqlNodeList(getPos());
+PartitionSpecCommaList(partSpec);
+}
+]
+{
+return new SqlAlterMat

(flink) 03/03: [FLINK-35189][test-utils] Introduce test-filesystem connector and catalog based on filesystem to support materialized table

2024-04-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 714d1cb2e0bd0df03393492dc87cbd800af63e1b
Author: fengli 
AuthorDate: Wed Apr 24 18:18:47 2024 +0800

[FLINK-35189][test-utils] Introduce test-filesystem connector and catalog 
based on filesystem to support materialized table
---
 .../flink-table-filesystem-test-utils/pom.xml  | 103 +++
 .../flink/connector/file/src/TestFileSource.java   | 189 ++
 .../file/table/TestFileSystemTableSource.java  |  88 +++
 .../file/testutils/TestFileSystemTableFactory.java |  54 ++
 .../file/testutils/catalog/JsonSerdeUtil.java  |  61 ++
 .../testutils/catalog/TestFileSystemCatalog.java   | 690 +
 .../catalog/TestFileSystemCatalogFactory.java  |  78 +++
 .../org.apache.flink.table.factories.Factory   |  17 +
 .../testutils/TestFileSystemTableFactoryTest.java  |  64 ++
 .../catalog/TestFileSystemCatalogFactoryTest.java  |  92 +++
 .../catalog/TestFileSystemCatalogITCase.java   |  89 +++
 .../catalog/TestFileSystemCatalogTest.java | 379 +++
 .../catalog/TestFileSystemCatalogTestBase.java |  56 ++
 .../src/test/resources/log4j2-test.properties  |  28 +
 flink-test-utils-parent/pom.xml|   1 +
 15 files changed, 1989 insertions(+)

diff --git a/flink-test-utils-parent/flink-table-filesystem-test-utils/pom.xml 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/pom.xml
new file mode 100644
index 000..e1c8fdce665
--- /dev/null
+++ b/flink-test-utils-parent/flink-table-filesystem-test-utils/pom.xml
@@ -0,0 +1,103 @@
+
+
+
+http://maven.apache.org/POM/4.0.0";
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+
+4.0.0
+
+
+org.apache.flink
+flink-test-utils-parent
+1.20-SNAPSHOT
+
+
+flink-table-filesystem-test-utils
+Flink : Test utils : Table Filesystem
+
+
+
+org.apache.flink
+flink-core
+${project.version}
+provided
+
+
+org.apache.flink
+flink-table-common
+${project.version}
+provided
+
+
+org.apache.flink
+flink-table-api-java-bridge
+${project.version}
+provided
+
+
+org.apache.flink
+flink-connector-files
+${project.version}
+provided
+
+
+
+
+org.apache.flink
+flink-table-common
+${project.version}
+tests
+test
+
+
+org.apache.flink
+flink-table-api-java
+${project.version}
+tests
+test
+
+
+org.apache.flink
+
flink-table-planner_${scala.binary.version}
+${project.version}
+test
+
+
+org.apache.flink
+
flink-table-planner_${scala.binary.version}
+${project.version}
+test
+test-jar
+
+
+org.apache.flink
+flink-csv
+${project.version}
+test
+
+
+org.apache.flink
+flink-test-utils
+${project.version}
+test
+
+
+
+
\ No newline at end of file
diff --git 
a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/src/TestFileSource.java
 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/src/TestFileSource.java
new file mode 100644
index 000..5d0e9ace77f
--- /dev/null
+++ 
b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/src/TestFileSource.java
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+packag

(flink) branch master updated (f64654e9749 -> 714d1cb2e0b)

2024-04-26 Thread ron
This is an automated email from the ASF dual-hosted git repository.

ron pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from f64654e9749 add HTTP options to java-storage client
 new 00d69e53fe7 [FLINK-35189][table-common] CatalogPropertiesUtil support 
serialize and deserialize materialized table
 new e3cda01cac7 [FLINK-35189][connectors/filesystem] Modify the visibility 
of filesystem connector related methods to protected
 new 714d1cb2e0b [FLINK-35189][test-utils] Introduce test-filesystem 
connector and catalog based on filesystem to support materialized table

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../connector/file/src/AbstractFileSource.java |  11 +-
 .../file/table/FileSystemTableFactory.java |   4 +-
 .../file/table/FileSystemTableSource.java  |   4 +-
 .../flink/table/catalog/CatalogPropertiesUtil.java | 140 -
 .../flink-table-filesystem-test-utils/pom.xml  | 103 +++
 .../flink/connector/file/src/TestFileSource.java   | 189 ++
 .../file/table/TestFileSystemTableSource.java  |  88 +++
 .../file/testutils/TestFileSystemTableFactory.java |  54 ++
 .../file/testutils/catalog/JsonSerdeUtil.java  |  61 ++
 .../testutils/catalog/TestFileSystemCatalog.java   | 690 +
 .../catalog/TestFileSystemCatalogFactory.java  |  78 +++
 .../org.apache.flink.table.factories.Factory   |  17 +
 .../testutils/TestFileSystemTableFactoryTest.java  |  64 ++
 .../catalog/TestFileSystemCatalogFactoryTest.java  |  92 +++
 .../catalog/TestFileSystemCatalogITCase.java   |  89 +++
 .../catalog/TestFileSystemCatalogTest.java | 379 +++
 .../catalog/TestFileSystemCatalogTestBase.java |  56 ++
 .../src/test/resources/log4j2-test.properties  |   0
 flink-test-utils-parent/pom.xml|   1 +
 19 files changed, 2111 insertions(+), 9 deletions(-)
 create mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/pom.xml
 create mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/src/TestFileSource.java
 create mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/connector/file/table/TestFileSystemTableSource.java
 create mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/TestFileSystemTableFactory.java
 create mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/catalog/JsonSerdeUtil.java
 create mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalog.java
 create mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalogFactory.java
 create mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
 create mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/src/test/java/org/apache/flink/table/file/testutils/TestFileSystemTableFactoryTest.java
 create mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/src/test/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalogFactoryTest.java
 create mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/src/test/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalogITCase.java
 create mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/src/test/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalogTest.java
 create mode 100644 
flink-test-utils-parent/flink-table-filesystem-test-utils/src/test/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalogTestBase.java
 copy {flink-table/flink-table-api-java => 
flink-test-utils-parent/flink-table-filesystem-test-utils}/src/test/resources/log4j2-test.properties
 (100%)



  1   2   >