You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/04/26 12:09:27 UTC

[GitHub] [flink] joemoe commented on a diff in pull request #19584: release notes for the 1.15 release

joemoe commented on code in PR #19584:
URL: https://github.com/apache/flink/pull/19584#discussion_r858635119


##########
docs/content/release-notes/flink-1.15.md:
##########
@@ -0,0 +1,379 @@
+---
+title: "Release Notes - Flink 1.15"
+---
+<!--
+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.
+-->
+
+# Release notes - Flink 1.15
+
+These release notes discuss important aspects, such as configuration, behavior, 
+or dependencies, that changed between Flink 1.14 and Flink 1.15. Please read these 
+notes carefully if you are planning to upgrade your Flink version to 1.15.
+
+## Summary of changed dependency names
+
+There are three changes in Flink 1.15 that require updating dependency names when 
+upgrading from earlier versions:
+
+* The newly introduced module `flink-table-planner-loader` (FLINK-25128) could 
+  replace the legacy `flink-table-planner_2.12`. As a consequence, `flink-table-uber` 
+  has been split into `flink-table-api-java-uber`, `flink-table-planner(-loader), 
+  and `table-runtime`. Besides, the artifactId of `flink-sql-client` has no Scala 
+  suffix (_2.11 / _2.12) anymore, and Scala users need to explicitly add a 
+  dependency to `flink-table-api-scala` or `flink-table-api-scala-bridge`.
+* Due to the efforts of removing Scala dependency from `flink-table-runtime` 
+  (FLINK-25114), the artifactId of `flink-table-runtime` has no Scala version 
+  suffix (\_2.11 / \_2.12) any more.
+* The FileSystem connector is no longer a part of the `flink-table-uber` module 
+  and changed to an optional dedicated `flink-connector-files` module 
+  (FLINK-24687). Besides, the artifactId of `flink-orc`, `flink-orc-nohive`, 
+  `flink-parquet` has no Scala suffix (\_2.11 / \_2.12) anymore.
+
+## JDK Upgrade
+
+The support of Java 8 is now deprecated and will be removed in a future release 
+([FLINK-25247](https://issues.apache.org/jira/browse/FLINK-25247)). We recommend 
+all users to migrate to Java 11.
+
+The default Java version in the Flink docker images is now Java 11 
+([FLINK-25251](https://issues.apache.org/jira/browse/FLINK-25251)). 
+There are images built with Java 8, tagged with “java8”. 
+
+## DataStream API
+
+### [TypeSerializer version mismatch during eagerly restore (FLINK-24858)](https://issues.apache.org/jira/browse/FLINK-24858)
+
+This ticket resolves an issue that during state migration between Flink versions
+the wrong serializer might have been picked.
+
+When upgrading from Flink 1.13.x please immediately choose 1.14.3 or higher and 
+skip 1.14.0, 1.14.1, 1.14.2 because all are affected and it might prevent your 
+job from starting.
+
+## Table API & SQL
+
+### [Make the legacy behavior disabled by default (FLINK-26551)](https://issues.apache.org/jira/browse/FLINK-26551)
+
+The legacy casting behavior has been disabled by default. This might have 
+implications on corner cases (string parsing, numeric overflows, to string 
+representation, varchar/binary precisions). Set 
+`table.exec.legacy-cast-behaviour=ENABLED` to restore the old behavior.
+
+### [Enforce CHAR/VARCHAR precision when outputting to a Sink (FLINK-24753)](https://issues.apache.org/jira/browse/FLINK-24753)
+
+`CHAR`/`VARCHAR` lengths are enforced (trimmed/padded) by default now before entering
+the table sink.
+
+### [Support the new type inference in Scala Table API table functions (FLINK-26518)](https://issues.apache.org/jira/browse/FLINK-26518)
+
+Table functions that are called using Scala implicit conversions have been updated 
+to use the new type system and new type inference. Users are requested to update 
+their UDFs or use the deprecated `TableEnvironment.registerFunction` to restore 
+the old behavior temporarily by calling the function via name.
+
+### [Propagate executor config to TableConfig (FLINK-26421)](https://issues.apache.org/jira/browse/FLINK-26421)
+
+`flink-conf.yaml` and other configurations from outer layers (e.g. CLI) are now 
+propagated into `TableConfig`. Even though configuration set directly in `TableConfig` 
+has still precedence, this change can have side effects if table configuration 
+was accidentally set in other layers.
+
+### [Remove pre FLIP-84 methods (FLINK-26090)](https://issues.apache.org/jira/browse/FLINK-26090)
+
+The previously deprecated methods `TableEnvironment.execute`, `Table.insertInto`,
+`TableEnvironment.fromTableSource`, `TableEnvironment.sqlUpdate`, and 
+`TableEnvironment.explain` have been removed. Please use the provided 
+alternatives introduced in 
+[FLIP-84](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=134745878).
+
+### [Fix parser generator warnings (FLINK-26053)](https://issues.apache.org/jira/browse/FLINK-26053)
+
+`STATEMENT` is a reserved keyword now. Use backticks to escape tables, fields and 
+other references.
+
+### [Expose uid generator for DataStream/Transformation providers (FLINK-25990)](https://issues.apache.org/jira/browse/FLINK-25990)
+
+`DataStreamScanProvider` and `DataStreamSinkProvider` for table connectors received 
+an additional method that might break implementations that used lambdas before. 
+We recommend static classes as a replacement and future robustness.
+
+### [Add new STATEMENT SET syntax (FLINK-25392)](https://issues.apache.org/jira/browse/FLINK-25392)
+
+It is recommended to update statement sets to the new SQL syntax: 
+
+```SQL
+EXECUTE STATEMENT SET BEGIN ... END;
+EXPLAIN STATEMENT SET BEGIN ... END;
+```
+
+### [Check & possible fix decimal precision and scale for all Aggregate functions (FLINK-24809)](https://issues.apache.org/jira/browse/FLINK-functions)
+
+This changes the result of a decimal `SUM()` with retraction and `AVG()` between
+1.14.0 and 1.14.1. It restores the behavior of 1.13 to be consistent with Hive/Spark.
+
+### [Clarify semantics of DecodingFormat and its data type (FLINK-24776)](https://issues.apache.org/jira/browse/FLINK-24776)
+
+The `DecodingFormat` interface was used for both projectable and non-projectable
+formats which led to inconsistent implementations. The `FileSystemTableSource` 
+has been updated to distinguish between those two interfaces now. Users that 
+implement custom formats for `FileSystemTableSource` might need to verify the 
+implementation and make sure to implement `ProjectableDecodingFormat` if necessary.
+
+### [Update user document for batch window tvf (FLINK-24760)](https://issues.apache.org/jira/browse/FLINK-24760)
+
+All window table-valued functions can now also be used in batch mode.
+
+### [Add user document for Window Deduplication (FLINK-24656)](https://issues.apache.org/jira/browse/FLINK-24656)
+
+Flink SQL now also supports Window Deduplication as a new window table-valued function.
+`WindowDeduplication` is a deduplication operator which removes rows that duplicate
+over a set of columns, keeping only the first one or the last one for each window
+and other partitioned keys.
+
+### [Push down partitions before filters (FLINK-24717)](https://issues.apache.org/jira/browse/FLINK-24717)
+
+This might have an impact on existing table source implementations as push down 
+filters might not contain partition predicates anymore. However, the connector 
+implementation for table sources that implement both partition and filter push 
+down became easier with this change.
+
+### [Flink SQL `SUM()` causes a precision error (FLINK-24691)](https://issues.apache.org/jira/browse/FLINK-24691)
+
+This changes the result of a decimal `SUM()` between 1.14.0 and 1.14.1. It restores 
+the behavior of 1.13 to be consistent with Hive/Spark.
+
+### [Use the new casting rules in TableResult#print (FLINK-24685)](https://issues.apache.org/jira/browse/FLINK-24685)
+
+The string representation of `BOOLEAN` columns from DDL results 
+(`true/false -> TRUE/FALSE`), and row columns in DQL results 
+(`+I[...] -> (...)`) has changed for printing.
+
+### [Sinks built with the unified sink framework do not receive timestamps when used in Table API (FLINK-24608)](https://issues.apache.org/jira/browse/FLINK-Table)
+
+This adds an additional operator to the topology if the new sink interfaces are used 
+(e.g. for Kafka). It could cause issues in 1.14.1 when restoring from a 1.14 savepoint. 
+A workaround is to cast the time attribute to a regular timestamp in the SQL statement
+closely before the sink.
+
+### [SQL functions should return `STRING` instead of `VARCHAR(2000)` (FLINK-24586)](https://issues.apache.org/jira/browse/FLINK-24586)
+
+Functions that returned `VARCHAR(2000)` in 1.14, return `VARCHAR` with maximum 
+length now. In particular this includes:
+
+```SQL
+SON_VALUE
+CHR
+REVERSE
+SPLIT_INDEX
+REGEXP_EXTRACT
+PARSE_URL
+FROM_UNIXTIME
+DECODE
+DATE_FORMAT
+CONVERT_TZ
+```
+
+## Connectors
+
+### [(Introducing GCS FileSystem Support and) Update GCS documentation (FLINK-25577)](https://issues.apache.org/jira/browse/FLINK-25577)
+
+The `flink-gs-fs-hadoop` FileSystem plugin for Google Cloud Storage (GCS) has been
+introduced. This allows Flink to read data from and write data to GCS via paths with 
+the `gs://` scheme, and it provides similar functionality for GCS as, for example, 
+the `flink-s3-fs-hadoop` provides for Amazon S3.
+
+In particular, this plugin supports the `RecoverableWriter` interface, which 
+allows it to be used with file sinks.
+
+Under the hood, the `flink-gs-fs-hadoop` uses Google's `gcs-connector` Hadoop 
+library for basic read/write operations, and it uses Google's `google-cloud-storage` 
+library to implement `RecoverableWriter` functionality.
+
+### [Remove MapR filesystem (FLINK-25553)](https://issues.apache.org/jira/browse/FLINK-25553)
+
+Support for the MapR FileSystem has been dropped.
+
+### [Make flink-connector-base dependency usage consistent across all connectors (FLINK-25927)](https://issues.apache.org/jira/browse/FLINK-25927)
+
+`flink-connector-base` was previously inconsistently used in connectors (directly 
+shaded in some and transitively pulled in via `flink-connector-files` which was 
+itself shaded in the table uber jar). 
+[FLINK-24687](https://issues.apache.org/jira/browse/FLINK-24687) moved 
+`flink-connector-files` 
+out of the `flink-table` uber jar. In order to address this, Flink 1.15 implements
+a combined approach for both making usage of `flink-connector-base` consistent 
+and ensuring a smooth transition for both Flink users and for external connector
+developers:
+
+1. all internal Flink connectors that depend on `flink-connector-base` now shade

Review Comment:
   would be good if you could elaborate @AHeise @afedulov.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org