DataStream API Integration

This page only discusses the integration with DataStream API in JVM languages such as Java or Scala. For Python, see the Python API area.

Both Table API and DataStream API are equally important when it comes to defining a data processing pipeline.

The DataStream API offers the primitives of stream processing (namely time, state, and dataflow management) in a relatively low-level imperative programming API. The Table API abstracts away many internals and provides a structured and declarative API.

Both APIs can work with bounded and unbounded streams.

Bounded streams need to be managed when processing historical data. Unbounded streams occur in real-time processing scenarios that might be initialized with historical data first.

For efficient execution, both APIs offer processing bounded streams in an optimized batch execution mode. However, since batch is just a special case of streaming, it is also possible to run pipelines of bounded streams in regular streaming execution mode.

Both DataStream API and Table API provide their own way of enabling the batch execution mode at the moment. In the near future, this will be further unified.

Pipelines in one API can be defined end-to-end without dependencies on the other API. However, it might be useful to mix both APIs for various reasons:

  • Use the table ecosystem for accessing catalogs or connecting to external systems easily, before implementing the main pipeline in DataStream API.
  • Access some of the SQL functions for stateless data normalization and cleansing, before implementing the main pipeline in DataStream API.
  • Switch to DataStream API every now and then if a more low-level operation (e.g. custom timer handling) is not present in Table API.

Flink provides special bridging functionalities to make the integration with DataStream API as smooth as possible.

Switching between DataStream and Table API adds some conversion overhead. For example, internal data structures of the table runtime (i.e. RowData) that partially work on binary data need to be converted to more user-friendly data structures (i.e. Row). Usually, this overhead can be neglected but is mentioned here for completeness.

Converting between DataStream and Table

Flink provides a specialized StreamTableEnvironment in Java and Scala for integrating with the DataStream API. Those environments extend the regular TableEnvironment with additional methods and take the StreamExecutionEnvironment used in the DataStream API as a parameter.

Currently, the StreamTableEnvironment does not support enabling the batch execution mode yet. Nevertheless, bounded streams can be processed there using the streaming execution mode but with lower efficiency.

Note, however, that the general TableEnvironment can work in both streaming execution or optimized batch execution mode.

The following code shows an example of how to go back and forth between the two APIs. Column names and types of the Table are automatically derived from the TypeInformation of the DataStream. Since the DataStream API does not support changelog processing natively, the code assumes append-only/insert-only semantics during the stream-to-table and table-to-stream conversion.

Java

  1. import org.apache.flink.streaming.api.datastream.DataStream;
  2. import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
  3. import org.apache.flink.table.api.Table;
  4. import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
  5. import org.apache.flink.types.Row;
  6. // create environments of both APIs
  7. StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
  8. StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env);
  9. // create a DataStream
  10. DataStream<String> dataStream = env.fromElements("Alice", "Bob", "John");
  11. // interpret the insert-only DataStream as a Table
  12. Table inputTable = tableEnv.fromDataStream(dataStream);
  13. // register the Table object as a view and query it
  14. tableEnv.createTemporaryView("InputTable", inputTable);
  15. Table resultTable = tableEnv.sqlQuery("SELECT UPPER(f0) FROM InputTable");
  16. // interpret the insert-only Table as a DataStream again
  17. DataStream<Row> resultStream = tableEnv.toDataStream(resultTable);
  18. // add a printing sink and execute in DataStream API
  19. resultStream.print();
  20. env.execute();
  21. // prints:
  22. // +I[Alice]
  23. // +I[Bob]
  24. // +I[John]

Scala

  1. import org.apache.flink.api.scala._
  2. import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
  3. import org.apache.flink.table.api.bridge.scala.StreamTableEnvironment
  4. // create environments of both APIs
  5. val env = StreamExecutionEnvironment.getExecutionEnvironment
  6. val tableEnv = StreamTableEnvironment.create(env)
  7. // create a DataStream
  8. val dataStream = env.fromElements("Alice", "Bob", "John")
  9. // interpret the insert-only DataStream as a Table
  10. val inputTable = tableEnv.fromDataStream(dataStream)
  11. // register the Table object as a view and query it
  12. tableEnv.createTemporaryView("InputTable", inputTable)
  13. val resultTable = tableEnv.sqlQuery("SELECT UPPER(f0) FROM InputTable")
  14. // interpret the insert-only Table as a DataStream again
  15. val resultStream = tableEnv.toDataStream(resultTable)
  16. // add a printing sink and execute in DataStream API
  17. resultStream.print()
  18. env.execute()
  19. // prints:
  20. // +I[Alice]
  21. // +I[Bob]
  22. // +I[John]

The complete semantics of fromDataStream and toDataStream can be found in the dedicated section below. In particular, the section discusses how to influence the schema derivation with more complex and nested types. It also covers working with event-time and watermarks.

Depending on the kind of query, in many cases the resulting dynamic table is a pipeline that does not only produce insert-only changes when coverting the Table to a DataStream but also produces retractions and other kinds of updates. During table-to-stream conversion, this could lead to an exception similar to

  1. Table sink 'Unregistered_DataStream_Sink_1' doesn't support consuming update changes [...].

in which case one needs to revise the query again or switch to toChangelogStream.

The following example shows how updating tables can be converted. Every result row represents an entry in a changelog with a change flag that can be queried by calling row.getKind() on it. In the example, the second score for Alice creates an update before (-U) and update after (+U) change.

Java

  1. import org.apache.flink.streaming.api.datastream.DataStream;
  2. import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
  3. import org.apache.flink.table.api.Table;
  4. import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
  5. import org.apache.flink.types.Row;
  6. // create environments of both APIs
  7. StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
  8. StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env);
  9. // create a DataStream
  10. DataStream<Row> dataStream = env.fromElements(
  11. Row.of("Alice", 12),
  12. Row.of("Bob", 10),
  13. Row.of("Alice", 100));
  14. // interpret the insert-only DataStream as a Table
  15. Table inputTable = tableEnv.fromDataStream(dataStream).as("name", "score");
  16. // register the Table object as a view and query it
  17. // the query contains an aggregation that produces updates
  18. tableEnv.createTemporaryView("InputTable", inputTable);
  19. Table resultTable = tableEnv.sqlQuery(
  20. "SELECT name, SUM(score) FROM InputTable GROUP BY name");
  21. // interpret the updating Table as a changelog DataStream
  22. DataStream<Row> resultStream = tableEnv.toChangelogStream(resultTable);
  23. // add a printing sink and execute in DataStream API
  24. resultStream.print();
  25. env.execute();
  26. // prints:
  27. // +I[Alice, 12]
  28. // +I[Bob, 10]
  29. // -U[Alice, 12]
  30. // +U[Alice, 112]

Scala

  1. import org.apache.flink.api.scala.typeutils.Types
  2. import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
  3. import org.apache.flink.table.api.bridge.scala.StreamTableEnvironment
  4. import org.apache.flink.types.Row
  5. // create environments of both APIs
  6. val env = StreamExecutionEnvironment.getExecutionEnvironment
  7. val tableEnv = StreamTableEnvironment.create(env)
  8. // create a DataStream
  9. val dataStream = env.fromElements(
  10. Row.of("Alice", Int.box(12)),
  11. Row.of("Bob", Int.box(10)),
  12. Row.of("Alice", Int.box(100))
  13. )(Types.ROW(Types.STRING, Types.INT))
  14. // interpret the insert-only DataStream as a Table
  15. val inputTable = tableEnv.fromDataStream(dataStream).as("name", "score")
  16. // register the Table object as a view and query it
  17. // the query contains an aggregation that produces updates
  18. tableEnv.createTemporaryView("InputTable", inputTable)
  19. val resultTable = tableEnv.sqlQuery("SELECT name, SUM(score) FROM InputTable GROUP BY name")
  20. // interpret the updating Table as a changelog DataStream
  21. val resultStream = tableEnv.toChangelogStream(resultTable)
  22. // add a printing sink and execute in DataStream API
  23. resultStream.print()
  24. env.execute()
  25. // prints:
  26. // +I[Alice, 12]
  27. // +I[Bob, 10]
  28. // -U[Alice, 12]
  29. // +U[Alice, 112]

The complete semantics of fromChangelogStream and toChangelogStream can be found in the dedicated section below. In particular, the section discusses how to influence the schema derivation with more complex and nested types. It covers working with event-time and watermarks. It discusses how to declare a primary key and changelog mode for the input and output streams.

Dependencies and Imports

Projects that combine Table API with DataStream API need to add one of the following bridging modules. They include transitive dependencies to flink-table-api-java or flink-table-api-scala and the corresponding language-specific DataStream API module.

Java

  1. <dependency>
  2. <groupId>org.apache.flink</groupId>
  3. <artifactId>flink-table-api-java-bridge_2.11</artifactId>
  4. <version>1.13.0</version>
  5. <scope>provided</scope>
  6. </dependency>

Scala

  1. <dependency>
  2. <groupId>org.apache.flink</groupId>
  3. <artifactId>flink-table-api-scala-bridge_2.11</artifactId>
  4. <version>1.13.0</version>
  5. <scope>provided</scope>
  6. </dependency>

The following imports are required to declare common pipelines using either the Java or Scala version of both DataStream API and Table API.

Java

  1. // imports for Java DataStream API
  2. import org.apache.flink.streaming.api.*;
  3. import org.apache.flink.streaming.api.environment.*;
  4. // imports for Table API with bridging to Java DataStream API
  5. import org.apache.flink.table.api.*;
  6. import org.apache.flink.table.api.bridge.java.*;

Scala

  1. // imports for Scala DataStream API
  2. import org.apache.flink.api.scala._
  3. import org.apache.flink.streaming.api.scala._
  4. // imports for Table API with bridging to Scala DataStream API
  5. import org.apache.flink.table.api._
  6. import org.apache.flink.table.api.bridge.scala._

Configuration

The TableEnvironment will adopt all configuration options from the passed StreamExecutionEnvironment. However, it cannot be guaranteed that further changes to the configuration of StreamExecutionEnvironment are propagated to the StreamTableEnvironment after its instantiation. Also, the reverse propagation of options from Table API to DataStream API is not supported.

We recommend setting all configuration options in DataStream API early before switching to Table API.

Java

  1. import java.time.ZoneId;
  2. import org.apache.flink.streaming.api.CheckpointingMode;
  3. import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
  4. import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
  5. // create Java DataStream API
  6. StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
  7. // set various configuration early
  8. env.setMaxParallelism(256);
  9. env.getConfig().addDefaultKryoSerializer(MyCustomType.class, CustomKryoSerializer.class);
  10. env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE);
  11. // then switch to Java Table API
  12. StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env);
  13. // set configuration early
  14. tableEnv.getConfig().setLocalTimeZone(ZoneId.of("Europe/Berlin"));
  15. // start defining your pipelines in both APIs...

Scala

  1. import java.time.ZoneId
  2. import org.apache.flink.api.scala._
  3. import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
  4. import org.apache.flink.streaming.api.CheckpointingMode
  5. import org.apache.flink.table.api.bridge.scala._
  6. // create Scala DataStream API
  7. val env = StreamExecutionEnvironment.getExecutionEnvironment
  8. // set various configuration early
  9. env.setMaxParallelism(256)
  10. env.getConfig.addDefaultKryoSerializer(classOf[MyCustomType], classOf[CustomKryoSerializer])
  11. env.getCheckpointConfig.setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE)
  12. // then switch to Scala Table API
  13. val tableEnv = StreamTableEnvironment.create(env)
  14. // set configuration early
  15. tableEnv.getConfig.setLocalTimeZone(ZoneId.of("Europe/Berlin"))
  16. // start defining your pipelines in both APIs...

Execution Behavior

Both APIs provide methods to execute pipelines. In other words: if requested, they compile a job graph that will be submitted to the cluster and triggered for execution. Results will be streamed to the declared sinks.

Usually, both APIs mark such behavior with the term execute in method names. However, the execution behavior is slightly different between Table API and DataStream API.

DataStream API

The DataStream API’s StreamExecutionEnvironment acts as a builder pattern to construct a complex pipeline. The pipeline possibly splits into multiple branches that might or might not end with a sink.

At least one sink must be defined. Otherwise, the following exception is thrown:

  1. java.lang.IllegalStateException: No operators defined in streaming topology. Cannot execute.

StreamExecutionEnvironment.execute() submits the entire constructed pipeline and clears the builder afterward. In other words: no sources and sinks are declared anymore, and a new pipeline can be added to the builder. Thus, every DataStream program usually ends with a call to StreamExecutionEnvironment.execute(). Alternatively, DataStream.executeAndCollect() implicitly defines a sink for streaming the results to the local client and only executes the current branch.

Table API

In the Table API, branching pipelines is only supported within a StatementSet where each branch must declare a final sink. Both TableEnvironment and also StreamTableEnvironment do not offer a dedicated general execute() method. Instead, they offer methods for submitting a single source-to-sink pipeline or a statement set:

  1. // execute with explicit sink
  2. tableEnv.from("InputTable").executeInsert("OutputTable")
  3. tableEnv.executeSql("INSERT INTO OutputTable SELECT * FROM InputTable")
  4. tableEnv.createStatementSet()
  5. .addInsert("OutputTable", tableEnv.from("InputTable"))
  6. .addInsert("OutputTable2", tableEnv.from("InputTable"))
  7. .execute()
  8. tableEnv.createStatementSet()
  9. .addInsertSql("INSERT INTO OutputTable SELECT * FROM InputTable")
  10. .addInsertSql("INSERT INTO OutputTable2 SELECT * FROM InputTable")
  11. .execute()
  12. // execute with implicit local sink
  13. tableEnv.from("InputTable").execute().print()
  14. tableEnv.executeSql("SELECT * FROM InputTable").print()

To combine both execution behaviors, every call to StreamTableEnvironment.toDataStream or StreamTableEnvironment.toChangelogStream will materialize (i.e. compile) the Table API sub-pipeline and insert it into the DataStream API pipeline builder. This means that StreamExecutionEnvironment.execute() or DataStream.executeAndCollect must be called afterwards. An execution in Table API will not trigger these “external parts”.

  1. // (1)
  2. // adds a branch with a printing sink to the StreamExecutionEnvironment
  3. tableEnv.toDataStream(table).print()
  4. // (2)
  5. // executes a Table API end-to-end pipeline as a Flink job and prints locally,
  6. // thus (1) has still not been executed
  7. table.execute().print()
  8. // executes the DataStream API pipeline with the sink defined in (1) as a
  9. // Flink job, (2) was already running before
  10. env.execute()

Handling of (Insert-Only) Streams

A StreamTableEnvironment offers the following methods to convert from and to DataStream API:

  • fromDataStream(DataStream): Interprets a stream of insert-only changes and arbitrary type as a table. Event-time and watermarks are not propagated by default.

  • fromDataStream(DataStream, Schema): Interprets a stream of insert-only changes and arbitrary type as a table. The optional schema allows to enrich column data types and add time attributes, watermarks strategies, other computed columns, or primary keys.

  • createTemporaryView(String, DataStream): Registers the stream under a name to access it in SQL. It is a shortcut for createTemporaryView(String, fromDataStream(DataStream)).

  • createTemporaryView(String, DataStream, Schema): Registers the stream under a name to access it in SQL. It is a shortcut for createTemporaryView(String, fromDataStream(DataStream, Schema)).

  • toDataStream(DataStream): Converts a table into a stream of insert-only changes. The default stream record type is org.apache.flink.types.Row. A single rowtime attribute column is written back into the DataStream API’s record. Watermarks are propagated as well.

  • toDataStream(DataStream, AbstractDataType): Converts a table into a stream of insert-only changes. This method accepts a data type to express the desired stream record type. The planner might insert implicit casts and reorders columns to map columns to fields of the (possibly nested) data type.

  • toDataStream(DataStream, Class): A shortcut for toDataStream(DataStream, DataTypes.of(Class)) to quickly create the desired data type reflectively.

From a Table API’s perspective, converting from and to DataStream API is similar to reading from or writing to a virtual table connector that has been defined using a CREATE TABLE DDL in SQL.

The schema part in the virtual CREATE TABLE name (schema) WITH (options) statement can be automatically derived from the DataStream’s type information, enriched, or entirely defined manually using org.apache.flink.table.api.Schema.

The virtual DataStream table connector exposes the following metadata for every row:

KeyData TypeDescriptionR/W
rowtimeTIMESTAMP_LTZ(3) NOT NULLStream record’s timestamp.R/W

The virtual DataStream table source implements SupportsSourceWatermark and thus allows calling the SOURCE_WATERMARK() built-in function as a watermark strategy to adopt watermarks from the DataStream API.

Examples for fromDataStream

The following code shows how to use fromDataStream for different scenarios.

Java

  1. import org.apache.flink.streaming.api.datastream.DataStream;
  2. import org.apache.flink.table.api.Schema;
  3. import org.apache.flink.table.api.Table;
  4. import java.time.Instant;
  5. // some example POJO
  6. public static class User {
  7. public String name;
  8. public Integer score;
  9. public Instant event_time;
  10. // default constructor for DataStream API
  11. public User() {}
  12. // fully assigning constructor for Table API
  13. public User(String name, Integer score, Instant event_time) {
  14. this.name = name;
  15. this.score = score;
  16. this.event_time = event_time;
  17. }
  18. }
  19. // create a DataStream
  20. DataStream<User> dataStream =
  21. env.fromElements(
  22. new User("Alice", 4, Instant.ofEpochMilli(1000)),
  23. new User("Bob", 6, Instant.ofEpochMilli(1001)),
  24. new User("Alice", 10, Instant.ofEpochMilli(1002)));
  25. // === EXAMPLE 1 ===
  26. // derive all physical columns automatically
  27. Table table = tableEnv.fromDataStream(dataStream);
  28. table.printSchema();
  29. // prints:
  30. // (
  31. // `name` STRING,
  32. // `score` INT,
  33. // `event_time` TIMESTAMP_LTZ(9)
  34. // )
  35. // === EXAMPLE 2 ===
  36. // derive all physical columns automatically
  37. // but add computed columns (in this case for creating a proctime attribute column)
  38. Table table = tableEnv.fromDataStream(
  39. dataStream,
  40. Schema.newBuilder()
  41. .columnByExpression("proc_time", "PROCTIME()")
  42. .build());
  43. table.printSchema();
  44. // prints:
  45. // (
  46. // `name` STRING,
  47. // `score` INT NOT NULL,
  48. // `event_time` TIMESTAMP_LTZ(9),
  49. // `proc_time` TIMESTAMP_LTZ(3) NOT NULL *PROCTIME* AS PROCTIME()
  50. //)
  51. // === EXAMPLE 3 ===
  52. // derive all physical columns automatically
  53. // but add computed columns (in this case for creating a rowtime attribute column)
  54. // and a custom watermark strategy
  55. Table table =
  56. tableEnv.fromDataStream(
  57. dataStream,
  58. Schema.newBuilder()
  59. .columnByExpression("rowtime", "CAST(event_time AS TIMESTAMP_LTZ(3))")
  60. .watermark("rowtime", "rowtime - INTERVAL '10' SECOND")
  61. .build());
  62. table.printSchema();
  63. // prints:
  64. // (
  65. // `name` STRING,
  66. // `score` INT,
  67. // `event_time` TIMESTAMP_LTZ(9),
  68. // `rowtime` TIMESTAMP_LTZ(3) *ROWTIME* AS CAST(event_time AS TIMESTAMP_LTZ(3)),
  69. // WATERMARK FOR `rowtime`: TIMESTAMP_LTZ(3) AS rowtime - INTERVAL '10' SECOND
  70. // )
  71. // === EXAMPLE 4 ===
  72. // derive all physical columns automatically
  73. // but access the stream record's timestamp for creating a rowtime attribute column
  74. // also rely on the watermarks generated in the DataStream API
  75. // we assume that a watermark strategy has been defined for `dataStream` before
  76. // (not part of this example)
  77. Table table =
  78. tableEnv.fromDataStream(
  79. dataStream,
  80. Schema.newBuilder()
  81. .columnByMetadata("rowtime", "TIMESTAMP_LTZ(3)")
  82. .watermark("rowtime", "SOURCE_WATERMARK()")
  83. .build());
  84. table.printSchema();
  85. // prints:
  86. // (
  87. // `name` STRING,
  88. // `score` INT,
  89. // `event_time` TIMESTAMP_LTZ(9),
  90. // `rowtime` TIMESTAMP_LTZ(3) *ROWTIME* METADATA,
  91. // WATERMARK FOR `rowtime`: TIMESTAMP_LTZ(3) AS SOURCE_WATERMARK()
  92. // )
  93. // === EXAMPLE 5 ===
  94. // define physical columns manually
  95. // in this example,
  96. // - we can reduce the default precision of timestamps from 9 to 3
  97. // - we also project the columns and put `event_time` to the beginning
  98. Table table =
  99. tableEnv.fromDataStream(
  100. dataStream,
  101. Schema.newBuilder()
  102. .column("event_time", "TIMESTAMP_LTZ(3)")
  103. .column("name", "STRING")
  104. .column("score", "INT")
  105. .watermark("event_time", "SOURCE_WATERMARK()")
  106. .build());
  107. table.printSchema();
  108. // prints:
  109. // (
  110. // `event_time` TIMESTAMP_LTZ(3) *ROWTIME*,
  111. // `name` VARCHAR(200),
  112. // `score` INT
  113. // )
  114. // note: the watermark strategy is not shown due to the inserted column reordering projection

Scala

  1. import org.apache.flink.api.scala._
  2. import java.time.Instant;
  3. // some example case class
  4. case class User(name: String, score: java.lang.Integer, event_time: java.time.Instant)
  5. // create a DataStream
  6. val dataStream = env.fromElements(
  7. User("Alice", 4, Instant.ofEpochMilli(1000)),
  8. User("Bob", 6, Instant.ofEpochMilli(1001)),
  9. User("Alice", 10, Instant.ofEpochMilli(1002)))
  10. // === EXAMPLE 1 ===
  11. // derive all physical columns automatically
  12. val table = tableEnv.fromDataStream(dataStream)
  13. table.printSchema()
  14. // prints:
  15. // (
  16. // `name` STRING,
  17. // `score` INT,
  18. // `event_time` TIMESTAMP_LTZ(9)
  19. // )
  20. // === EXAMPLE 2 ===
  21. // derive all physical columns automatically
  22. // but add computed columns (in this case for creating a proctime attribute column)
  23. val table = tableEnv.fromDataStream(
  24. dataStream,
  25. Schema.newBuilder()
  26. .columnByExpression("proc_time", "PROCTIME()")
  27. .build())
  28. table.printSchema()
  29. // prints:
  30. // (
  31. // `name` STRING,
  32. // `score` INT NOT NULL,
  33. // `event_time` TIMESTAMP_LTZ(9),
  34. // `proc_time` TIMESTAMP_LTZ(3) NOT NULL *PROCTIME* AS PROCTIME()
  35. //)
  36. // === EXAMPLE 3 ===
  37. // derive all physical columns automatically
  38. // but add computed columns (in this case for creating a rowtime attribute column)
  39. // and a custom watermark strategy
  40. val table =
  41. tableEnv.fromDataStream(
  42. dataStream,
  43. Schema.newBuilder()
  44. .columnByExpression("rowtime", "CAST(event_time AS TIMESTAMP_LTZ(3))")
  45. .watermark("rowtime", "rowtime - INTERVAL '10' SECOND")
  46. .build())
  47. table.printSchema()
  48. // prints:
  49. // (
  50. // `name` STRING,
  51. // `score` INT,
  52. // `event_time` TIMESTAMP_LTZ(9),
  53. // `rowtime` TIMESTAMP_LTZ(3) *ROWTIME* AS CAST(event_time AS TIMESTAMP_LTZ(3)),
  54. // WATERMARK FOR `rowtime`: TIMESTAMP_LTZ(3) AS rowtime - INTERVAL '10' SECOND
  55. // )
  56. // === EXAMPLE 4 ===
  57. // derive all physical columns automatically
  58. // but access the stream record's timestamp for creating a rowtime attribute column
  59. // also rely on the watermarks generated in the DataStream API
  60. // we assume that a watermark strategy has been defined for `dataStream` before
  61. // (not part of this example)
  62. val table =
  63. tableEnv.fromDataStream(
  64. dataStream,
  65. Schema.newBuilder()
  66. .columnByMetadata("rowtime", "TIMESTAMP_LTZ(3)")
  67. .watermark("rowtime", "SOURCE_WATERMARK()")
  68. .build())
  69. table.printSchema()
  70. // prints:
  71. // (
  72. // `name` STRING,
  73. // `score` INT,
  74. // `event_time` TIMESTAMP_LTZ(9),
  75. // `rowtime` TIMESTAMP_LTZ(3) *ROWTIME* METADATA,
  76. // WATERMARK FOR `rowtime`: TIMESTAMP_LTZ(3) AS SOURCE_WATERMARK()
  77. // )
  78. // === EXAMPLE 5 ===
  79. // define physical columns manually
  80. // in this example,
  81. // - we can reduce the default precision of timestamps from 9 to 3
  82. // - we also project the columns and put `event_time` to the beginning
  83. val table =
  84. tableEnv.fromDataStream(
  85. dataStream,
  86. Schema.newBuilder()
  87. .column("event_time", "TIMESTAMP_LTZ(3)")
  88. .column("name", "STRING")
  89. .column("score", "INT")
  90. .watermark("event_time", "SOURCE_WATERMARK()")
  91. .build())
  92. table.printSchema()
  93. // prints:
  94. // (
  95. // `event_time` TIMESTAMP_LTZ(3) *ROWTIME*,
  96. // `name` VARCHAR(200),
  97. // `score` INT
  98. // )
  99. // note: the watermark strategy is not shown due to the inserted column reordering projection

Example 1 illustrates a simple use case when no time-based operations are needed.

Example 4 is the most common use case when time-based operations such as windows or interval joins should be part of the pipeline. Example 2 is the most common use case when these time-based operations should work in processing time.

Example 5 entirely relies on the declaration of the user. This can be useful to replace generic types from the DataStream API (which would be RAW in the Table API) with proper data types.

Since DataType is richer than TypeInformation, we can easily enable immutable POJOs and other complex data structures. The following example in Java shows what is possible. Check also the Data Types & Serialization page of the DataStream API for more information about the supported types there.

  1. import org.apache.flink.streaming.api.datastream.DataStream;
  2. import org.apache.flink.table.api.DataTypes;
  3. import org.apache.flink.table.api.Schema;
  4. import org.apache.flink.table.api.Table;
  5. // the DataStream API does not support immutable POJOs yet,
  6. // the class will result in a generic type that is a RAW type in Table API by default
  7. public static class User {
  8. public final String name;
  9. public final Integer score;
  10. public User(String name, Integer score) {
  11. this.name = name;
  12. this.score = score;
  13. }
  14. }
  15. // create a DataStream
  16. DataStream<User> dataStream = env.fromElements(
  17. new User("Alice", 4),
  18. new User("Bob", 6),
  19. new User("Alice", 10));
  20. // since fields of a RAW type cannot be accessed, every stream record is treated as an atomic type
  21. // leading to a table with a single column `f0`
  22. Table table = tableEnv.fromDataStream(dataStream);
  23. table.printSchema();
  24. // prints:
  25. // (
  26. // `f0` RAW('User', '...')
  27. // )
  28. // instead, declare a more useful data type for columns using the Table API's type system
  29. // in a custom schema and rename the columns in a following `as` projection
  30. Table table = tableEnv
  31. .fromDataStream(
  32. dataStream,
  33. Schema.newBuilder()
  34. .column("f0", DataTypes.of(User.class))
  35. .build())
  36. .as("user");
  37. table.printSchema();
  38. // prints:
  39. // (
  40. // `user` *User<`name` STRING,`score` INT>*
  41. // )
  42. // data types can be extracted reflectively as above or explicitly defined
  43. Table table3 = tableEnv
  44. .fromDataStream(
  45. dataStream,
  46. Schema.newBuilder()
  47. .column(
  48. "f0",
  49. DataTypes.STRUCTURED(
  50. User.class,
  51. DataTypes.FIELD("name", DataTypes.STRING()),
  52. DataTypes.FIELD("score", DataTypes.INT())))
  53. .build())
  54. .as("user");
  55. table.printSchema();
  56. // prints:
  57. // (
  58. // `user` *User<`name` STRING,`score` INT>*
  59. // )

Examples for createTemporaryView

A DataStream can be registered directly as a view (possibly enriched with a schema).

Views created from a DataStream can only be registered as temporary views. Due to their inline/anonymous nature, it is not possible to register them in a permanent catalog.

The following code shows how to use createTemporaryView for different scenarios.

Java

  1. import org.apache.flink.api.java.tuple.Tuple2;
  2. import org.apache.flink.streaming.api.datastream.DataStream;
  3. // create some DataStream
  4. DataStream<Tuple2<Long, String>> dataStream = env.fromElements(
  5. Tuple2.of(12L, "Alice"),
  6. Tuple2.of(0L, "Bob"));
  7. // === EXAMPLE 1 ===
  8. // register the DataStream as view "MyView" in the current session
  9. // all columns are derived automatically
  10. tableEnv.createTemporaryView("MyView", dataStream);
  11. tableEnv.from("MyView").printSchema();
  12. // prints:
  13. // (
  14. // `f0` BIGINT NOT NULL,
  15. // `f1` STRING
  16. // )
  17. // === EXAMPLE 2 ===
  18. // register the DataStream as view "MyView" in the current session,
  19. // provide a schema to adjust the columns similar to `fromDataStream`
  20. // in this example, the derived NOT NULL information has been removed
  21. tableEnv.createTemporaryView(
  22. "MyView",
  23. dataStream,
  24. Schema.newBuilder()
  25. .column("f0", "BIGINT")
  26. .column("f1", "STRING")
  27. .build());
  28. tableEnv.from("MyView").printSchema();
  29. // prints:
  30. // (
  31. // `f0` BIGINT,
  32. // `f1` STRING
  33. // )
  34. // === EXAMPLE 3 ===
  35. // use the Table API before creating the view if it is only about renaming columns
  36. tableEnv.createTemporaryView(
  37. "MyView",
  38. tableEnv.fromDataStream(dataStream).as("id", "name"));
  39. tableEnv.from("MyView").printSchema();
  40. // prints:
  41. // (
  42. // `id` BIGINT NOT NULL,
  43. // `name` STRING
  44. // )

Scala

  1. // create some DataStream
  2. val dataStream: DataStream[(Long, String)] = env.fromElements(
  3. (12L, "Alice"),
  4. (0L, "Bob"))
  5. // === EXAMPLE 1 ===
  6. // register the DataStream as view "MyView" in the current session
  7. // all columns are derived automatically
  8. tableEnv.createTemporaryView("MyView", dataStream)
  9. tableEnv.from("MyView").printSchema()
  10. // prints:
  11. // (
  12. // `_1` BIGINT NOT NULL,
  13. // `_2` STRING
  14. // )
  15. // === EXAMPLE 2 ===
  16. // register the DataStream as view "MyView" in the current session,
  17. // provide a schema to adjust the columns similar to `fromDataStream`
  18. // in this example, the derived NOT NULL information has been removed
  19. tableEnv.createTemporaryView(
  20. "MyView",
  21. dataStream,
  22. Schema.newBuilder()
  23. .column("_1", "BIGINT")
  24. .column("_2", "STRING")
  25. .build())
  26. tableEnv.from("MyView").printSchema()
  27. // prints:
  28. // (
  29. // `_1` BIGINT,
  30. // `_2` STRING
  31. // )
  32. // === EXAMPLE 3 ===
  33. // use the Table API before creating the view if it is only about renaming columns
  34. tableEnv.createTemporaryView(
  35. "MyView",
  36. tableEnv.fromDataStream(dataStream).as("id", "name"))
  37. tableEnv.from("MyView").printSchema()
  38. // prints:
  39. // (
  40. // `id` BIGINT NOT NULL,
  41. // `name` STRING
  42. // )

Examples for toDataStream

The following code shows how to use toDataStream for different scenarios.

Java

  1. import org.apache.flink.streaming.api.datastream.DataStream;
  2. import org.apache.flink.table.api.DataTypes;
  3. import org.apache.flink.table.api.Table;
  4. import org.apache.flink.types.Row;
  5. import java.time.Instant;
  6. // POJO with mutable fields
  7. // since no fully assigning constructor is defined, the field order
  8. // is alphabetical [event_time, name, score]
  9. public static class User {
  10. public String name;
  11. public Integer score;
  12. public Instant event_time;
  13. }
  14. tableEnv.executeSql(
  15. "CREATE TABLE GeneratedTable "
  16. + "("
  17. + " name STRING,"
  18. + " score INT,"
  19. + " event_time TIMESTAMP_LTZ(3),"
  20. + " WATERMARK FOR event_time AS event_time - INTERVAL '10' SECOND"
  21. + ")"
  22. + "WITH ('connector'='datagen')");
  23. Table table = tableEnv.from("GeneratedTable");
  24. // === EXAMPLE 1 ===
  25. // use the default conversion to instances of Row
  26. // since `event_time` is a single rowtime attribute, it is inserted into the DataStream
  27. // metadata and watermarks are propagated
  28. DataStream<Row> dataStream = tableEnv.toDataStream(table);
  29. // === EXAMPLE 2 ===
  30. // a data type is extracted from class `User`,
  31. // the planner reorders fields and inserts implicit casts where possible to convert internal
  32. // data structures to the desired structured type
  33. // since `event_time` is a single rowtime attribute, it is inserted into the DataStream
  34. // metadata and watermarks are propagated
  35. DataStream<User> dataStream = tableEnv.toDataStream(table, User.class);
  36. // data types can be extracted reflectively as above or explicitly defined
  37. DataStream<User> dataStream =
  38. tableEnv.toDataStream(
  39. table,
  40. DataTypes.STRUCTURED(
  41. User.class,
  42. DataTypes.FIELD("name", DataTypes.STRING()),
  43. DataTypes.FIELD("score", DataTypes.INT()),
  44. DataTypes.FIELD("event_time", DataTypes.TIMESTAMP_LTZ(3))));

Scala

  1. import org.apache.flink.streaming.api.scala.DataStream
  2. import org.apache.flink.table.api.DataTypes
  3. case class User(name: String, score: java.lang.Integer, event_time: java.time.Instant)
  4. tableEnv.executeSql(
  5. "CREATE TABLE GeneratedTable "
  6. + "("
  7. + " name STRING,"
  8. + " score INT,"
  9. + " event_time TIMESTAMP_LTZ(3),"
  10. + " WATERMARK FOR event_time AS event_time - INTERVAL '10' SECOND"
  11. + ")"
  12. + "WITH ('connector'='datagen')")
  13. val table = tableEnv.from("GeneratedTable")
  14. // === EXAMPLE 1 ===
  15. // use the default conversion to instances of Row
  16. // since `event_time` is a single rowtime attribute, it is inserted into the DataStream
  17. // metadata and watermarks are propagated
  18. val dataStream: DataStream[Row] = tableEnv.toDataStream(table)
  19. // === EXAMPLE 2 ===
  20. // a data type is extracted from class `User`,
  21. // the planner reorders fields and inserts implicit casts where possible to convert internal
  22. // data structures to the desired structured type
  23. // since `event_time` is a single rowtime attribute, it is inserted into the DataStream
  24. // metadata and watermarks are propagated
  25. val dataStream: DataStream[User] = tableEnv.toDataStream(table, User.class)
  26. // data types can be extracted reflectively as above or explicitly defined
  27. val dataStream: DataStream[User] =
  28. tableEnv.toDataStream(
  29. table,
  30. DataTypes.STRUCTURED(
  31. User.class,
  32. DataTypes.FIELD("name", DataTypes.STRING()),
  33. DataTypes.FIELD("score", DataTypes.INT()),
  34. DataTypes.FIELD("event_time", DataTypes.TIMESTAMP_LTZ(3))))

Note that only non-updating tables are supported by toDataStream. Usually, time-based operations such as windows, interval joins, or the MATCH_RECOGNIZE clause are a good fit for insert-only pipelines next to simple operations like projections and filters. Pipelines with operations that produce updates can use toChangelogStream.

Handling of Changelog Streams

Internally, Flink’s table runtime is a changelog processor. The concepts page describes how dynamic tables and streams relate to each other.

A StreamTableEnvironment offers the following methods to expose these change data capture (CDC) functionalities:

  • fromChangelogStream(DataStream): Interprets a stream of changelog entries as a table. The stream record type must be org.apache.flink.types.Row since its RowKind flag is evaluated during runtime. Event-time and watermarks are not propagated by default. This method expects a changelog containing all kinds of changes (enumerated in org.apache.flink.types.RowKind) as the default ChangelogMode.

  • fromChangelogStream(DataStream, Schema): Allows to define a schema for the DataStream similar to fromDataStream(DataStream, Schema). Otherwise the semantics are equal to fromChangelogStream(DataStream).

  • fromChangelogStream(DataStream, Schema, ChangelogMode): Gives full control about how to interpret a stream as a changelog. The passed ChangelogMode helps the planner to distinguish between insert-only, upsert, or retract behavior.

  • toChangelogStream(Table): Reverse operation of fromChangelogStream(DataStream). It produces a stream with instances of org.apache.flink.types.Row and sets the RowKind flag for every record at runtime. All kinds of updating tables are supported by this method. If the input table contains a single rowtime column, it will be propagated into a stream record’s timestamp. Watermarks will be propagated as well.

  • toChangelogStream(Table, Schema): Reverse operation of fromChangelogStream(DataStream, Schema). The method can enrich the produced column data types. The planner might insert implicit casts if necessary. It is possible to write out the rowtime as a metadata column.

  • toChangelogStream(Table, Schema, ChangelogMode): Gives full control about how to convert a table to a changelog stream. The passed ChangelogMode helps the planner to distinguish between insert-only, upsert, or retract behavior.

From a Table API’s perspective, converting from and to DataStream API is similar to reading from or writing to a virtual table connector that has been defined using a CREATE TABLE DDL in SQL.

Because fromChangelogStream behaves similar to fromDataStream, we recommend reading the previous section before continuing here.

This virtual connector also supports reading and writing the rowtime metadata of the stream record.

The virtual table source implements SupportsSourceWatermark.

Examples for fromChangelogStream

The following code shows how to use fromChangelogStream for different scenarios.

Java

  1. import org.apache.flink.streaming.api.datastream.DataStream;
  2. import org.apache.flink.table.api.Schema;
  3. import org.apache.flink.table.api.Table;
  4. import org.apache.flink.table.connector.ChangelogMode;
  5. import org.apache.flink.types.Row;
  6. import org.apache.flink.types.RowKind;
  7. // === EXAMPLE 1 ===
  8. // interpret the stream as a retract stream
  9. // create a changelog DataStream
  10. DataStream<Row> dataStream =
  11. env.fromElements(
  12. Row.ofKind(RowKind.INSERT, "Alice", 12),
  13. Row.ofKind(RowKind.INSERT, "Bob", 5),
  14. Row.ofKind(RowKind.UPDATE_BEFORE, "Alice", 12),
  15. Row.ofKind(RowKind.UPDATE_AFTER, "Alice", 100));
  16. // interpret the DataStream as a Table
  17. Table table = tableEnv.fromChangelogStream(dataStream);
  18. // register the table under a name and perform an aggregation
  19. tableEnv.createTemporaryView("InputTable", table);
  20. tableEnv
  21. .executeSql("SELECT f0 AS name, SUM(f1) AS score FROM InputTable GROUP BY f0")
  22. .print();
  23. // prints:
  24. // +----+--------------------------------+-------------+
  25. // | op | name | score |
  26. // +----+--------------------------------+-------------+
  27. // | +I | Bob | 5 |
  28. // | +I | Alice | 12 |
  29. // | -D | Alice | 12 |
  30. // | +I | Alice | 100 |
  31. // +----+--------------------------------+-------------+
  32. // === EXAMPLE 2 ===
  33. // interpret the stream as an upsert stream (without a need for UPDATE_BEFORE)
  34. // create a changelog DataStream
  35. DataStream<Row> dataStream =
  36. env.fromElements(
  37. Row.ofKind(RowKind.INSERT, "Alice", 12),
  38. Row.ofKind(RowKind.INSERT, "Bob", 5),
  39. Row.ofKind(RowKind.UPDATE_AFTER, "Alice", 100));
  40. // interpret the DataStream as a Table
  41. Table table =
  42. tableEnv.fromChangelogStream(
  43. dataStream,
  44. Schema.newBuilder().primaryKey("f0").build(),
  45. ChangelogMode.upsert());
  46. // register the table under a name and perform an aggregation
  47. tableEnv.createTemporaryView("InputTable", table);
  48. tableEnv
  49. .executeSql("SELECT f0 AS name, SUM(f1) AS score FROM InputTable GROUP BY f0")
  50. .print();
  51. // prints:
  52. // +----+--------------------------------+-------------+
  53. // | op | name | score |
  54. // +----+--------------------------------+-------------+
  55. // | +I | Bob | 5 |
  56. // | +I | Alice | 12 |
  57. // | -D | Alice | 12 |
  58. // | +I | Alice | 100 |
  59. // +----+--------------------------------+-------------+

Scala

  1. import org.apache.flink.api.scala.typeutils.Types
  2. import org.apache.flink.table.api.Schema
  3. import org.apache.flink.table.connector.ChangelogMode
  4. import org.apache.flink.types.{Row, RowKind}
  5. // === EXAMPLE 1 ===
  6. // interpret the stream as a retract stream
  7. // create a changelog DataStream
  8. val dataStream = env.fromElements(
  9. Row.ofKind(RowKind.INSERT, "Alice", Int.box(12)),
  10. Row.ofKind(RowKind.INSERT, "Bob", Int.box(5)),
  11. Row.ofKind(RowKind.UPDATE_BEFORE, "Alice", Int.box(12)),
  12. Row.ofKind(RowKind.UPDATE_AFTER, "Alice", Int.box(100))
  13. )(Types.ROW(Types.STRING, Types.INT))
  14. // interpret the DataStream as a Table
  15. val table = tableEnv.fromChangelogStream(dataStream)
  16. // register the table under a name and perform an aggregation
  17. tableEnv.createTemporaryView("InputTable", table)
  18. tableEnv
  19. .executeSql("SELECT f0 AS name, SUM(f1) AS score FROM InputTable GROUP BY f0")
  20. .print()
  21. // prints:
  22. // +----+--------------------------------+-------------+
  23. // | op | name | score |
  24. // +----+--------------------------------+-------------+
  25. // | +I | Bob | 5 |
  26. // | +I | Alice | 12 |
  27. // | -D | Alice | 12 |
  28. // | +I | Alice | 100 |
  29. // +----+--------------------------------+-------------+
  30. // === EXAMPLE 2 ===
  31. // interpret the stream as an upsert stream (without a need for UPDATE_BEFORE)
  32. // create a changelog DataStream
  33. val dataStream = env.fromElements(
  34. Row.ofKind(RowKind.INSERT, "Alice", Int.box(12)),
  35. Row.ofKind(RowKind.INSERT, "Bob", Int.box(5)),
  36. Row.ofKind(RowKind.UPDATE_AFTER, "Alice", Int.box(100))
  37. )(Types.ROW(Types.STRING, Types.INT))
  38. // interpret the DataStream as a Table
  39. val table =
  40. tableEnv.fromChangelogStream(
  41. dataStream,
  42. Schema.newBuilder().primaryKey("f0").build(),
  43. ChangelogMode.upsert())
  44. // register the table under a name and perform an aggregation
  45. tableEnv.createTemporaryView("InputTable", table)
  46. tableEnv
  47. .executeSql("SELECT f0 AS name, SUM(f1) AS score FROM InputTable GROUP BY f0")
  48. .print()
  49. // prints:
  50. // +----+--------------------------------+-------------+
  51. // | op | name | score |
  52. // +----+--------------------------------+-------------+
  53. // | +I | Bob | 5 |
  54. // | +I | Alice | 12 |
  55. // | -D | Alice | 12 |
  56. // | +I | Alice | 100 |
  57. // +----+--------------------------------+-------------+

The default ChangelogMode shown in example 1 should be sufficient for most use cases as it accepts all kinds of changes.

However, example 2 shows how to limit the kinds of incoming changes for efficiency by reducing the number of update messages by 50%.

Examples for toChangelogStream

The following code shows how to use toChangelogStream for different scenarios.

Java

  1. import org.apache.flink.streaming.api.datastream.DataStream;
  2. import org.apache.flink.streaming.api.functions.ProcessFunction;
  3. import org.apache.flink.table.api.DataTypes;
  4. import org.apache.flink.table.api.Schema;
  5. import org.apache.flink.table.api.Table;
  6. import org.apache.flink.table.data.StringData;
  7. import org.apache.flink.types.Row;
  8. import org.apache.flink.util.Collector;
  9. import static org.apache.flink.table.api.Expressions.*;
  10. // create Table with event-time
  11. tableEnv.executeSql(
  12. "CREATE TABLE GeneratedTable "
  13. + "("
  14. + " name STRING,"
  15. + " score INT,"
  16. + " event_time TIMESTAMP_LTZ(3),"
  17. + " WATERMARK FOR event_time AS event_time - INTERVAL '10' SECOND"
  18. + ")"
  19. + "WITH ('connector'='datagen')");
  20. Table table = tableEnv.from("GeneratedTable");
  21. // === EXAMPLE 1 ===
  22. // convert to DataStream in the simplest and most general way possible (no event-time)
  23. Table simpleTable = tableEnv
  24. .fromValues(row("Alice", 12), row("Alice", 2), row("Bob", 12))
  25. .as("name", "score")
  26. .groupBy($("name"))
  27. .select($("name"), $("score").sum());
  28. tableEnv
  29. .toChangelogStream(simpleTable)
  30. .executeAndCollect()
  31. .forEachRemaining(System.out::println);
  32. // prints:
  33. // +I[Bob, 12]
  34. // +I[Alice, 12]
  35. // -U[Alice, 12]
  36. // +U[Alice, 14]
  37. // === EXAMPLE 2 ===
  38. // convert to DataStream in the simplest and most general way possible (with event-time)
  39. DataStream<Row> dataStream = tableEnv.toChangelogStream(table);
  40. // since `event_time` is a single time attribute in the schema, it is set as the
  41. // stream record's timestamp by default; however, at the same time, it remains part of the Row
  42. dataStream.process(
  43. new ProcessFunction<Row, Void>() {
  44. @Override
  45. public void processElement(Row row, Context ctx, Collector<Void> out) {
  46. // prints: [name, score, event_time]
  47. System.out.println(row.getFieldNames(true));
  48. // timestamp exists twice
  49. assert ctx.timestamp() == row.<Instant>getFieldAs("event_time").toEpochMilli();
  50. }
  51. });
  52. env.execute();
  53. // === EXAMPLE 3 ===
  54. // convert to DataStream but write out the time attribute as a metadata column which means
  55. // it is not part of the physical schema anymore
  56. DataStream<Row> dataStream = tableEnv.toChangelogStream(
  57. table,
  58. Schema.newBuilder()
  59. .column("name", "STRING")
  60. .column("score", "INT")
  61. .columnByMetadata("rowtime", "TIMESTAMP_LTZ(3)")
  62. .build());
  63. // the stream record's timestamp is defined by the metadata; it is not part of the Row
  64. dataStream.process(
  65. new ProcessFunction<Row, Void>() {
  66. @Override
  67. public void processElement(Row row, Context ctx, Collector<Void> out) {
  68. // prints: [name, score]
  69. System.out.println(row.getFieldNames(true));
  70. // timestamp exists once
  71. System.out.println(ctx.timestamp());
  72. }
  73. });
  74. env.execute();
  75. // === EXAMPLE 4 ===
  76. // for advanced users, it is also possible to use more internal data structures for efficiency
  77. // note that this is only mentioned here for completeness because using internal data structures
  78. // adds complexity and additional type handling
  79. // however, converting a TIMESTAMP_LTZ column to `Long` or STRING to `byte[]` might be convenient,
  80. // also structured types can be represented as `Row` if needed
  81. DataStream<Row> dataStream = tableEnv.toChangelogStream(
  82. table,
  83. Schema.newBuilder()
  84. .column(
  85. "name",
  86. DataTypes.STRING().bridgedTo(StringData.class))
  87. .column(
  88. "score",
  89. DataTypes.INT())
  90. .column(
  91. "event_time",
  92. DataTypes.TIMESTAMP_LTZ(3).bridgedTo(Long.class))
  93. .build());
  94. // leads to a stream of Row(name: StringData, score: Integer, event_time: Long)

Scala

  1. import org.apache.flink.api.scala._
  2. import org.apache.flink.streaming.api.functions.ProcessFunction
  3. import org.apache.flink.streaming.api.scala.DataStream
  4. import org.apache.flink.table.api._
  5. import org.apache.flink.types.Row
  6. import org.apache.flink.util.Collector
  7. import java.time.Instant
  8. // create Table with event-time
  9. tableEnv.executeSql(
  10. "CREATE TABLE GeneratedTable "
  11. + "("
  12. + " name STRING,"
  13. + " score INT,"
  14. + " event_time TIMESTAMP_LTZ(3),"
  15. + " WATERMARK FOR event_time AS event_time - INTERVAL '10' SECOND"
  16. + ")"
  17. + "WITH ('connector'='datagen')")
  18. val table = tableEnv.from("GeneratedTable")
  19. // === EXAMPLE 1 ===
  20. // convert to DataStream in the simplest and most general way possible (no event-time)
  21. val simpleTable = tableEnv
  22. .fromValues(row("Alice", 12), row("Alice", 2), row("Bob", 12))
  23. .as("name", "score")
  24. .groupBy($"name")
  25. .select($"name", $"score".sum())
  26. tableEnv
  27. .toChangelogStream(simpleTable)
  28. .executeAndCollect()
  29. .foreach(println)
  30. // prints:
  31. // +I[Bob, 12]
  32. // +I[Alice, 12]
  33. // -U[Alice, 12]
  34. // +U[Alice, 14]
  35. // === EXAMPLE 2 ===
  36. // convert to DataStream in the simplest and most general way possible (with event-time)
  37. val dataStream: DataStream[Row] = tableEnv.toChangelogStream(table)
  38. // since `event_time` is a single time attribute in the schema, it is set as the
  39. // stream record's timestamp by default; however, at the same time, it remains part of the Row
  40. dataStream.process(new ProcessFunction[Row, Unit] {
  41. override def processElement(
  42. row: Row,
  43. ctx: ProcessFunction[Row, Unit]#Context,
  44. out: Collector[Unit]): Unit = {
  45. // prints: [name, score, event_time]
  46. println(row.getFieldNames(true))
  47. // timestamp exists twice
  48. assert(ctx.timestamp() == row.getFieldAs[Instant]("event_time").toEpochMilli)
  49. }
  50. })
  51. env.execute()
  52. // === EXAMPLE 3 ===
  53. // convert to DataStream but write out the time attribute as a metadata column which means
  54. // it is not part of the physical schema anymore
  55. val dataStream: DataStream[Row] = tableEnv.toChangelogStream(
  56. table,
  57. Schema.newBuilder()
  58. .column("name", "STRING")
  59. .column("score", "INT")
  60. .columnByMetadata("rowtime", "TIMESTAMP_LTZ(3)")
  61. .build())
  62. // the stream record's timestamp is defined by the metadata; it is not part of the Row
  63. dataStream.process(new ProcessFunction[Row, Unit] {
  64. override def processElement(
  65. row: Row,
  66. ctx: ProcessFunction[Row, Unit]#Context,
  67. out: Collector[Unit]): Unit = {
  68. // prints: [name, score]
  69. println(row.getFieldNames(true))
  70. // timestamp exists once
  71. println(ctx.timestamp())
  72. }
  73. })
  74. env.execute()
  75. // === EXAMPLE 4 ===
  76. // for advanced users, it is also possible to use more internal data structures for better
  77. // efficiency
  78. // note that this is only mentioned here for completeness because using internal data structures
  79. // adds complexity and additional type handling
  80. // however, converting a TIMESTAMP_LTZ column to `Long` or STRING to `byte[]` might be convenient,
  81. // also structured types can be represented as `Row` if needed
  82. val dataStream: DataStream[Row] = tableEnv.toChangelogStream(
  83. table,
  84. Schema.newBuilder()
  85. .column(
  86. "name",
  87. DataTypes.STRING().bridgedTo(classOf[StringData]))
  88. .column(
  89. "score",
  90. DataTypes.INT())
  91. .column(
  92. "event_time",
  93. DataTypes.TIMESTAMP_LTZ(3).bridgedTo(class[Long]))
  94. .build())
  95. // leads to a stream of Row(name: StringData, score: Integer, event_time: Long)

For more information about which conversions are supported for data types in Example 4, see the Table API’s Data Types page.

The behavior of toChangelogStream(Table).executeAndCollect() is equal to calling Table.execute().collect(). However, toChangelogStream(Table) might be more useful for tests because it allows to access the produced watermarks in a subsequent ProcessFunction in DataStream API.

Mapping between TypeInformation and DataType

The DataStream API uses instances of org.apache.flink.api.common.typeinfo.TypeInformation to describe the record type that travels in the stream. In particular, it defines how to serialize and deserialize records from one DataStream operator to the other. It also helps in serializing state into savepoints and checkpoints.

The Table API uses custom data structures to represent records internally and exposes org.apache.flink.table.types.DataType to users for declaring the external format into which the data structures are converted for easier usage in sources, sinks, UDFs, or DataStream API.

DataType is richer than TypeInformation as it also includes details about the logical SQL type. Therefore, some details will be added implicitly during the conversion.

Column names and types of a Table are automatically derived from the TypeInformation of the DataStream. Use DataStream.getType() to check whether the type information has been detected correctly via the DataStream API’s reflective type extraction facilities. If the outermost record’s TypeInformation is a CompositeType, it will be flattened in the first level when deriving a table’s schema.

TypeInformation to DataType

The following rules apply when converting TypeInformation to a DataType:

  • All subclasses of TypeInformation are mapped to logical types, including nullability that is aligned with Flink’s built-in serializers.

  • Subclasses of TupleTypeInfoBase are translated into a row (for Row) or structured type (for tuples, POJOs, and case classes).

  • BigDecimal is converted to DECIMAL(38, 18) by default.

  • The order of PojoTypeInfo fields is determined by a constructor with all fields as its parameters. If that is not found during the conversion, the field order will be alphabetical.

  • GenericTypeInfo and other TypeInformation that cannot be represented as one of the listed org.apache.flink.table.api.DataTypes will be treated as a black-box RAW type. The current session configuration is used to materialize the serializer of the raw type. Composite nested fields will not be accessible then.

  • See TypeInfoDataTypeConverter for the full translation logic.

Use DataTypes.of(TypeInformation) to call the above logic in custom schema declaration or in UDFs.

DataType to TypeInformation

The table runtime will make sure to properly serialize the output records to the first operator of the DataStream API.

Afterward, the type information semantics of the DataStream API need to be considered.

Legacy Conversion

The following section describes outdated parts of the API that will be removed in future versions.

In particular, these parts might not be well integrated into many recent new features and refactorings (e.g. RowKind is not correctly set, type systems don’t integrate smoothly).

将 DataStream 或 DataSet 转换成表

与在 TableEnvironment 注册 DataStreamDataSet 不同,DataStream 和 DataSet 还可以直接转换成 Table。如果你想在 Table API 的查询中使用表,这将非常便捷。

Java

  1. // get StreamTableEnvironment
  2. // registration of a DataSet in a BatchTableEnvironment is equivalent
  3. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  4. DataStream<Tuple2<Long, String>> stream = ...
  5. // Convert the DataStream into a Table with default fields "f0", "f1"
  6. Table table1 = tableEnv.fromDataStream(stream);
  7. // Convert the DataStream into a Table with fields "myLong", "myString"
  8. Table table2 = tableEnv.fromDataStream(stream, $("myLong"), $("myString"));

Scala

  1. // get TableEnvironment
  2. // registration of a DataSet is equivalent
  3. val tableEnv = ... // see "Create a TableEnvironment" section
  4. val stream: DataStream[(Long, String)] = ...
  5. // convert the DataStream into a Table with default fields "_1", "_2"
  6. val table1: Table = tableEnv.fromDataStream(stream)
  7. // convert the DataStream into a Table with fields "myLong", "myString"
  8. val table2: Table = tableEnv.fromDataStream(stream, $"myLong", $"myString")

将表转换成 DataStream 或 DataSet

Table 可以被转换成 DataStreamDataSet。通过这种方式,定制的 DataSet 或 DataStream 程序就可以在 Table API 或者 SQL 的查询结果上运行了。

Table 转换为 DataStream 或者 DataSet 时,你需要指定生成的 DataStream 或者 DataSet 的数据类型,即,Table 的每行数据要转换成的数据类型。通常最方便的选择是转换成 Row 。以下列表概述了不同选项的功能:

  • Row: 字段按位置映射,字段数量任意,支持 null 值,无类型安全(type-safe)检查。
  • POJO: 字段按名称映射(POJO 必须按Table 中字段名称命名),字段数量任意,支持 null 值,无类型安全检查。
  • Case Class: 字段按位置映射,不支持 null 值,有类型安全检查。
  • Tuple: 字段按位置映射,字段数量少于 22(Scala)或者 25(Java),不支持 null 值,无类型安全检查。
  • Atomic Type: Table 必须有一个字段,不支持 null 值,有类型安全检查。

将表转换成 DataStream

流式查询(streaming query)的结果表会动态更新,即,当新纪录到达查询的输入流时,查询结果会改变。因此,像这样将动态查询结果转换成 DataStream 需要对表的更新方式进行编码。

Table 转换为 DataStream 有两种模式:

  1. Append Mode: 仅当动态 Table 仅通过INSERT更改进行修改时,才可以使用此模式,即,它仅是追加操作,并且之前输出的结果永远不会更新。
  2. Retract Mode: 任何情形都可以使用此模式。它使用 boolean 值对 INSERTDELETE 操作的数据进行标记。

    Java

  1. // get StreamTableEnvironment.
  2. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. // Table with two fields (String name, Integer age)
  4. Table table = ...
  5. // convert the Table into an append DataStream of Row by specifying the class
  6. DataStream<Row> dsRow = tableEnv.toAppendStream(table, Row.class);
  7. // convert the Table into an append DataStream of Tuple2<String, Integer>
  8. // via a TypeInformation
  9. TupleTypeInfo<Tuple2<String, Integer>> tupleType = new TupleTypeInfo<>(
  10. Types.STRING(),
  11. Types.INT());
  12. DataStream<Tuple2<String, Integer>> dsTuple =
  13. tableEnv.toAppendStream(table, tupleType);
  14. // convert the Table into a retract DataStream of Row.
  15. // A retract stream of type X is a DataStream<Tuple2<Boolean, X>>.
  16. // The boolean field indicates the type of the change.
  17. // True is INSERT, false is DELETE.
  18. DataStream<Tuple2<Boolean, Row>> retractStream =
  19. tableEnv.toRetractStream(table, Row.class);

Scala

  1. // get TableEnvironment.
  2. // registration of a DataSet is equivalent
  3. val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" section
  4. // Table with two fields (String name, Integer age)
  5. val table: Table = ...
  6. // convert the Table into an append DataStream of Row
  7. val dsRow: DataStream[Row] = tableEnv.toAppendStream[Row](table)
  8. // convert the Table into an append DataStream of Tuple2[String, Int]
  9. val dsTuple: DataStream[(String, Int)] dsTuple =
  10. tableEnv.toAppendStream[(String, Int)](table)
  11. // convert the Table into a retract DataStream of Row.
  12. // A retract stream of type X is a DataStream[(Boolean, X)].
  13. // The boolean field indicates the type of the change.
  14. // True is INSERT, false is DELETE.
  15. val retractStream: DataStream[(Boolean, Row)] = tableEnv.toRetractStream[Row](table)

注意: 文档动态表给出了有关动态表及其属性的详细讨论。

注意 一旦 Table 被转化为 DataStream,必须使用 StreamExecutionEnvironment 的 execute 方法执行该 DataStream 作业。

将表转换成 DataSet

Table 转换成 DataSet 的过程如下:

Java

  1. // get BatchTableEnvironment
  2. BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env);
  3. // Table with two fields (String name, Integer age)
  4. Table table = ...
  5. // convert the Table into a DataSet of Row by specifying a class
  6. DataSet<Row> dsRow = tableEnv.toDataSet(table, Row.class);
  7. // convert the Table into a DataSet of Tuple2<String, Integer> via a TypeInformation
  8. TupleTypeInfo<Tuple2<String, Integer>> tupleType = new TupleTypeInfo<>(
  9. Types.STRING(),
  10. Types.INT());
  11. DataSet<Tuple2<String, Integer>> dsTuple =
  12. tableEnv.toDataSet(table, tupleType);

Scala

  1. // get TableEnvironment
  2. // registration of a DataSet is equivalent
  3. val tableEnv = BatchTableEnvironment.create(env)
  4. // Table with two fields (String name, Integer age)
  5. val table: Table = ...
  6. // convert the Table into a DataSet of Row
  7. val dsRow: DataSet[Row] = tableEnv.toDataSet[Row](table)
  8. // convert the Table into a DataSet of Tuple2[String, Int]
  9. val dsTuple: DataSet[(String, Int)] = tableEnv.toDataSet[(String, Int)](table)

注意 一旦 Table 被转化为 DataSet,必须使用 ExecutionEnvironment 的 execute 方法执行该 DataSet 作业。

数据类型到 Table Schema 的映射

Flink 的 DataStream 和 DataSet APIs 支持多样的数据类型。例如 Tuple(Scala 内置以及Flink Java tuple)、POJO 类型、Scala case class 类型以及 Flink 的 Row 类型等允许嵌套且有多个可在表的表达式中访问的字段的复合数据类型。其他类型被视为原子类型。下面,我们讨论 Table API 如何将这些数据类型类型转换为内部 row 表示形式,并提供将 DataStream 转换成 Table 的样例。

数据类型到 table schema 的映射有两种方式:基于字段位置基于字段名称

基于位置映射

基于位置的映射可在保持字段顺序的同时为字段提供更有意义的名称。这种映射方式可用于具有特定的字段顺序的复合数据类型以及原子类型。如 tuple、row 以及 case class 这些复合数据类型都有这样的字段顺序。然而,POJO 类型的字段则必须通过名称映射(参见下一章)。可以将字段投影出来,但不能使用as重命名。

定义基于位置的映射时,输入数据类型中一定不能存在指定的名称,否则 API 会假定应该基于字段名称进行映射。如果未指定任何字段名称,则使用默认的字段名称和复合数据类型的字段顺序,或者使用 f0 表示原子类型。

Java

  1. // get a StreamTableEnvironment, works for BatchTableEnvironment equivalently
  2. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section;
  3. DataStream<Tuple2<Long, Integer>> stream = ...
  4. // convert DataStream into Table with field "myLong" only
  5. Table table = tableEnv.fromDataStream(stream, $("myLong"));
  6. // convert DataStream into Table with field names "myLong" and "myInt"
  7. Table table = tableEnv.fromDataStream(stream, $("myLong"), $("myInt"));

Scala

  1. // get a TableEnvironment
  2. val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" section
  3. val stream: DataStream[(Long, Int)] = ...
  4. // convert DataStream into Table with field "myLong" only
  5. val table: Table = tableEnv.fromDataStream(stream, $"myLong")
  6. // convert DataStream into Table with field names "myLong" and "myInt"
  7. val table: Table = tableEnv.fromDataStream(stream, $"myLong", $"myInt")

基于名称的映射

基于名称的映射适用于任何数据类型包括 POJO 类型。这是定义 table schema 映射最灵活的方式。映射中的所有字段均按名称引用,并且可以通过 as 重命名。字段可以被重新排序和映射。

若果没有指定任何字段名称,则使用默认的字段名称和复合数据类型的字段顺序,或者使用 f0 表示原子类型。

Java

  1. // get a StreamTableEnvironment, works for BatchTableEnvironment equivalently
  2. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. DataStream<Tuple2<Long, Integer>> stream = ...
  4. // convert DataStream into Table with field "f1" only
  5. Table table = tableEnv.fromDataStream(stream, $("f1"));
  6. // convert DataStream into Table with swapped fields
  7. Table table = tableEnv.fromDataStream(stream, $("f1"), $("f0"));
  8. // convert DataStream into Table with swapped fields and field names "myInt" and "myLong"
  9. Table table = tableEnv.fromDataStream(stream, $("f1").as("myInt"), $("f0").as("myLong"));

Scala

  1. // get a TableEnvironment
  2. val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" section
  3. val stream: DataStream[(Long, Int)] = ...
  4. // convert DataStream into Table with field "_2" only
  5. val table: Table = tableEnv.fromDataStream(stream, $"_2")
  6. // convert DataStream into Table with swapped fields
  7. val table: Table = tableEnv.fromDataStream(stream, $"_2", $"_1")
  8. // convert DataStream into Table with swapped fields and field names "myInt" and "myLong"
  9. val table: Table = tableEnv.fromDataStream(stream, $"_2" as "myInt", $"_1" as "myLong")

原子类型

Flink 将基础数据类型(IntegerDoubleString)或者通用数据类型(不可再拆分的数据类型)视为原子类型。原子类型的 DataStream 或者 DataSet 会被转换成只有一条属性的 Table。属性的数据类型可以由原子类型推断出,还可以重新命名属性。

Java

  1. // get a StreamTableEnvironment, works for BatchTableEnvironment equivalently
  2. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. DataStream<Long> stream = ...
  4. // convert DataStream into Table with field name "myLong"
  5. Table table = tableEnv.fromDataStream(stream, $("myLong"));

Scala

  1. // get a TableEnvironment
  2. val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" section
  3. val stream: DataStream[Long] = ...
  4. // convert DataStream into Table with field name "myLong"
  5. val table: Table = tableEnv.fromDataStream(stream, $"myLong")

Tuple类型(Scala 和 Java)和 Case Class类型(仅 Scala)

Flink 支持 Scala 的内置 tuple 类型并给 Java 提供自己的 tuple 类型。两种 tuple 的 DataStream 和 DataSet 都能被转换成表。可以通过提供所有字段名称来重命名字段(基于位置映射)。如果没有指明任何字段名称,则会使用默认的字段名称。如果引用了原始字段名称(对于 Flink tuple 为f0f1 … …,对于 Scala tuple 为_1_2 … …),则 API 会假定映射是基于名称的而不是基于位置的。基于名称的映射可以通过 as 对字段和投影进行重新排序。

Java

  1. // get a StreamTableEnvironment, works for BatchTableEnvironment equivalently
  2. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. DataStream<Tuple2<Long, String>> stream = ...
  4. // convert DataStream into Table with renamed field names "myLong", "myString" (position-based)
  5. Table table = tableEnv.fromDataStream(stream, $("myLong"), $("myString"));
  6. // convert DataStream into Table with reordered fields "f1", "f0" (name-based)
  7. Table table = tableEnv.fromDataStream(stream, $("f1"), $("f0"));
  8. // convert DataStream into Table with projected field "f1" (name-based)
  9. Table table = tableEnv.fromDataStream(stream, $("f1"));
  10. // convert DataStream into Table with reordered and aliased fields "myString", "myLong" (name-based)
  11. Table table = tableEnv.fromDataStream(stream, $("f1").as("myString"), $("f0").as("myLong"));

Scala

  1. // get a TableEnvironment
  2. val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" section
  3. val stream: DataStream[(Long, String)] = ...
  4. // convert DataStream into Table with field names "myLong", "myString" (position-based)
  5. val table: Table = tableEnv.fromDataStream(stream, $"myLong", $"myString")
  6. // convert DataStream into Table with reordered fields "_2", "_1" (name-based)
  7. val table: Table = tableEnv.fromDataStream(stream, $"_2", $"_1")
  8. // convert DataStream into Table with projected field "_2" (name-based)
  9. val table: Table = tableEnv.fromDataStream(stream, $"_2")
  10. // convert DataStream into Table with reordered and aliased fields "myString", "myLong" (name-based)
  11. val table: Table = tableEnv.fromDataStream(stream, $"_2" as "myString", $"_1" as "myLong")
  12. // define case class
  13. case class Person(name: String, age: Int)
  14. val streamCC: DataStream[Person] = ...
  15. // convert DataStream into Table with field names 'myName, 'myAge (position-based)
  16. val table = tableEnv.fromDataStream(streamCC, $"myName", $"myAge")
  17. // convert DataStream into Table with reordered and aliased fields "myAge", "myName" (name-based)
  18. val table: Table = tableEnv.fromDataStream(stream, $"age" as "myAge", $"name" as "myName")

POJO 类型 (Java 和 Scala)

Flink 支持 POJO 类型作为复合类型。确定 POJO 类型的规则记录在这里.

在不指定字段名称的情况下将 POJO 类型的 DataStreamDataSet 转换成 Table 时,将使用原始 POJO 类型字段的名称。名称映射需要原始名称,并且不能按位置进行。字段可以使用别名(带有 as 关键字)来重命名,重新排序和投影。

Java

  1. // get a StreamTableEnvironment, works for BatchTableEnvironment equivalently
  2. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. // Person is a POJO with fields "name" and "age"
  4. DataStream<Person> stream = ...
  5. // convert DataStream into Table with renamed fields "myAge", "myName" (name-based)
  6. Table table = tableEnv.fromDataStream(stream, $("age").as("myAge"), $("name").as("myName"));
  7. // convert DataStream into Table with projected field "name" (name-based)
  8. Table table = tableEnv.fromDataStream(stream, $("name"));
  9. // convert DataStream into Table with projected and renamed field "myName" (name-based)
  10. Table table = tableEnv.fromDataStream(stream, $("name").as("myName"));

Scala

  1. // get a TableEnvironment
  2. val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" section
  3. // Person is a POJO with field names "name" and "age"
  4. val stream: DataStream[Person] = ...
  5. // convert DataStream into Table with renamed fields "myAge", "myName" (name-based)
  6. val table: Table = tableEnv.fromDataStream(stream, $"age" as "myAge", $"name" as "myName")
  7. // convert DataStream into Table with projected field "name" (name-based)
  8. val table: Table = tableEnv.fromDataStream(stream, $"name")
  9. // convert DataStream into Table with projected and renamed field "myName" (name-based)
  10. val table: Table = tableEnv.fromDataStream(stream, $"name" as "myName")

Row类型

Row 类型支持任意数量的字段以及具有 null 值的字段。字段名称可以通过 RowTypeInfo 指定,也可以在将 RowDataStreamDataSet 转换为 Table 时指定。Row 类型的字段映射支持基于名称和基于位置两种方式。字段可以通过提供所有字段的名称的方式重命名(基于位置映射)或者分别选择进行投影/排序/重命名(基于名称映射)。

Java

  1. // get a StreamTableEnvironment, works for BatchTableEnvironment equivalently
  2. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. // DataStream of Row with two fields "name" and "age" specified in `RowTypeInfo`
  4. DataStream<Row> stream = ...
  5. // convert DataStream into Table with renamed field names "myName", "myAge" (position-based)
  6. Table table = tableEnv.fromDataStream(stream, $("myName"), $("myAge"));
  7. // convert DataStream into Table with renamed fields "myName", "myAge" (name-based)
  8. Table table = tableEnv.fromDataStream(stream, $("name").as("myName"), $("age").as("myAge"));
  9. // convert DataStream into Table with projected field "name" (name-based)
  10. Table table = tableEnv.fromDataStream(stream, $("name"));
  11. // convert DataStream into Table with projected and renamed field "myName" (name-based)
  12. Table table = tableEnv.fromDataStream(stream, $("name").as("myName"));

Scala

  1. // get a TableEnvironment
  2. val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" section
  3. // DataStream of Row with two fields "name" and "age" specified in `RowTypeInfo`
  4. val stream: DataStream[Row] = ...
  5. // convert DataStream into Table with renamed field names "myName", "myAge" (position-based)
  6. val table: Table = tableEnv.fromDataStream(stream, $"myName", $"myAge")
  7. // convert DataStream into Table with renamed fields "myName", "myAge" (name-based)
  8. val table: Table = tableEnv.fromDataStream(stream, $"name" as "myName", $"age" as "myAge")
  9. // convert DataStream into Table with projected field "name" (name-based)
  10. val table: Table = tableEnv.fromDataStream(stream, $"name")
  11. // convert DataStream into Table with projected and renamed field "myName" (name-based)
  12. val table: Table = tableEnv.fromDataStream(stream, $"name" as "myName")