diff --git a/docs/content.zh/docs/connectors/table/datagen.md b/docs/content.zh/docs/connectors/table/datagen.md index 7337df5e6baf6..96dff1c11d99c 100644 --- a/docs/content.zh/docs/connectors/table/datagen.md +++ b/docs/content.zh/docs/connectors/table/datagen.md @@ -129,7 +129,7 @@ CREATE TABLE datagen ( 可选 0 Duration - 随机生成器生成相对当前时间向过去偏移的最大值,适用于 Timestamp 类型. + 随机生成器生成相对当前时间向过去偏移的最大值,适用于 timestamp 类型。
fields.#.length
diff --git a/docs/content/docs/connectors/table/datagen.md b/docs/content/docs/connectors/table/datagen.md index 865cd7c5342ea..d4f9432aa9f30 100644 --- a/docs/content/docs/connectors/table/datagen.md +++ b/docs/content/docs/connectors/table/datagen.md @@ -159,7 +159,7 @@ Types random Resolves a past timestamp relative to the current timestamp of the local machine. - The max past is specified by the 'max-past' option. + The max past can be specified by the 'max-past' option. @@ -167,7 +167,7 @@ Types random Resolves a past timestamp relative to the current timestamp of the local machine. - The max past is specified by the 'max-past' option. + The max past can be specified by the 'max-past' option. @@ -264,7 +264,7 @@ Connector Options optional 0 Duration - Maximum past of timestamp random generator, work for timestamp types. + Maximum past of timestamp random generator, only works for timestamp types.
fields.#.length
diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/connector/datagen/table/DataGenVisitorBase.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/connector/datagen/table/DataGenVisitorBase.java index ba8c281808410..8fbc894e9966c 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/connector/datagen/table/DataGenVisitorBase.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/connector/datagen/table/DataGenVisitorBase.java @@ -24,13 +24,9 @@ import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.streaming.api.functions.source.datagen.DataGenerator; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.types.logical.DateType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.TimeType; -import org.apache.flink.table.types.logical.TimestampType; -import org.apache.flink.table.types.logical.ZonedTimestampType; import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor; import java.io.Serializable; @@ -66,24 +62,6 @@ public DataGeneratorContainer visit(TimeType timeType) { return DataGeneratorContainer.of(TimeGenerator.of(() -> LocalTime.now().get(MILLI_OF_DAY))); } - @Override - public DataGeneratorContainer visit(TimestampType timestampType) { - return DataGeneratorContainer.of( - TimeGenerator.of(() -> TimestampData.fromEpochMillis(System.currentTimeMillis()))); - } - - @Override - public DataGeneratorContainer visit(ZonedTimestampType zonedTimestampType) { - return DataGeneratorContainer.of( - TimeGenerator.of(() -> TimestampData.fromEpochMillis(System.currentTimeMillis()))); - } - - @Override - public DataGeneratorContainer visit(LocalZonedTimestampType localZonedTimestampType) { - return DataGeneratorContainer.of( - TimeGenerator.of(() -> TimestampData.fromEpochMillis(System.currentTimeMillis()))); - } - @Override protected DataGeneratorContainer defaultMethod(LogicalType logicalType) { throw new ValidationException("Unsupported type: " + logicalType);