Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-21569] Upgrade flink-shaded-jackson version to 2.12.1-13.0 #16642

Merged
merged 2 commits into from
Aug 3, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,14 @@

package org.apache.flink.formats.csv;

import org.apache.flink.table.api.config.ExecutionConfigOptions;
import org.apache.flink.table.planner.runtime.batch.sql.BatchFileSystemITCaseBase;
import org.apache.flink.table.planner.runtime.utils.BatchTestBase;
import org.apache.flink.types.Row;
import org.apache.flink.util.CollectionUtil;
import org.apache.flink.util.FileUtils;

import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.runners.Enclosed;
import org.junit.runner.RunWith;
Expand Down Expand Up @@ -90,4 +94,70 @@ public void testEscapeChar() throws Exception {
Arrays.asList(Row.of("x5,5,1,1"), Row.of("x5,5,2,2")));
}
}

/**
* IT case which checks for a bug in Jackson 2.10. When the 4000th character in csv file is the
* new line character (\n) an exception will be thrown. After upgrading jackson to >= 2.11 this
* bug should not exist anymore.
*/
public static class JacksonVersionUpgradeITCase extends BatchTestBase {

@Test
public void testCsvFileWithNewLineAt4000() throws Exception {
StringBuilder csvContent = new StringBuilder("# ");
for (int i = 0; i < 97; i++) {
csvContent.append("-");
}
csvContent.append("\n");
for (int i = 0; i < 50; i++) {
for (int j = 0; j < 49; j++) {
csvContent.append("a");
}
csvContent.append(",");
for (int j = 0; j < 49; j++) {
csvContent.append("b");
}
csvContent.append("\n");
}

File tempCsvFile = File.createTempFile("new-line-at-4000", ".csv");
tempCsvFile.createNewFile();
FileUtils.writeFileUtf8(tempCsvFile, csvContent.toString());

tEnv().getConfig()
.getConfiguration()
.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
tEnv().executeSql(
"CREATE TABLE T (\n"
+ " a VARCHAR,\n"
+ " b VARCHAR\n"
+ ") WITH (\n"
+ " 'connector' = 'filesystem',\n"
+ " 'path' = 'file://"
+ tempCsvFile.toString()
+ "',\n"
+ " 'format' = 'csv',\n"
+ " 'csv.allow-comments' = 'true'\n"
+ ")")
.await();
List<Row> results =
CollectionUtil.iteratorToList(
tEnv().executeSql("SELECT a, b FROM T").collect());

Assert.assertEquals(50, results.size());
for (Row actual : results) {
StringBuilder a = new StringBuilder();
for (int i = 0; i < 49; i++) {
a.append("a");
}
StringBuilder b = new StringBuilder();
for (int i = 0; i < 49; i++) {
b.append("b");
}
Assert.assertEquals(2, actual.getArity());
Assert.assertEquals(a.toString(), actual.getField(0));
Assert.assertEquals(b.toString(), actual.getField(1));
}
}
}
}
6 changes: 3 additions & 3 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ under the License.
<zookeeper.version>3.4.14</zookeeper.version>
<!-- Only the curator2 TestingServer works with ZK 3.4 -->
<curator.version>2.12.0</curator.version>
<jackson.version>2.10.1</jackson.version>
<jackson.version>2.12.1</jackson.version>
<prometheus.version>0.8.1</prometheus.version>
<avro.version>1.10.0</avro.version>
<javax.activation.api.version>1.2.0</javax.activation.api.version>
Expand Down Expand Up @@ -285,13 +285,13 @@ under the License.
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-jackson</artifactId>
<version>${jackson.version}-${flink.shaded.version}</version>
<version>${jackson.version}-13.0</version>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-jackson-module-jsonSchema</artifactId>
<version>${jackson.version}-${flink.shaded.version}</version>
<version>${jackson.version}-13.0</version>
</dependency>

<dependency>
Expand Down