Skip to content
This repository has been archived by the owner on Dec 14, 2022. It is now read-only.

fix flink pulsar table in json schema bug #476

Open
wants to merge 1 commit into
base: release-1.9
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import java.util.Map;
import java.util.TimeZone;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Function;

/**
* Various options for decoding a JSON record.
Expand All @@ -48,7 +47,6 @@ public class JSONOptions implements Serializable {
private final String columnNameOfCorruptRecord;
private final boolean dropFieldIfAllNull;
private final ConcurrentHashMap<String, TimeZone> computedTimeZones;
private final Function<String, TimeZone> computeTimeZone;
private final TimeZone timeZone;
private final FastDateFormat dateFormat;
private final FastDateFormat timestampFormat;
Expand Down Expand Up @@ -125,7 +123,6 @@ public JSONOptions(
}

this.computedTimeZones = new ConcurrentHashMap<>();
this.computeTimeZone = timezoneId -> TimeZone.getTimeZone(timezoneId);

this.timeZone = getTimeZone(parameters.getOrDefault("timezone", defaultTimeZoneId));

Expand All @@ -139,8 +136,8 @@ protected String checkEncoding(String enc) {
return enc;
}

private TimeZone getTimeZone(String timeZoneeId) {
return computedTimeZones.computeIfAbsent(timeZoneeId, computeTimeZone);
private TimeZone getTimeZone(String timeZoneId) {
return computedTimeZones.computeIfAbsent(timeZoneId, TimeZone::getTimeZone);
}

/** Sets config options on a Jackson [[JsonFactory]]. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,19 +30,18 @@

import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
import java.text.ParseException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.function.BiFunction;
import java.util.function.Function;

/**
* JSON record parser.
*/
@Slf4j
public class JacksonRecordParser {
public class JacksonRecordParser implements Serializable{

private final DataType schema;

Expand Down Expand Up @@ -558,7 +557,7 @@ public Object parseJsonToken(JsonParser parser, DataType dataType, PartialFunc f
}
}

interface PartialFunc {
interface PartialFunc extends Serializable{
boolean isDefinedAt(JsonToken token);

Object apply(JsonToken token);
Expand All @@ -579,11 +578,19 @@ default Object applyOrElse(JsonToken token, JsonParser parser, DataType dataType
}
}

interface BiFunctionWithException<T, U, R> {
interface Function<T, R> extends Serializable{
R apply(T t);
}

interface BiFunction<T, U, R> extends Serializable{
R apply(T t, U u);
}

interface BiFunctionWithException<T, U, R> extends Serializable{
R apply(T t, U u) throws BadRecordException;
}

static class FailureSafeRecordParser {
static class FailureSafeRecordParser implements Serializable {
private final BiFunctionWithException<String, Row, Row> rawParser;
private final ParseMode mode;
private final FieldsDataType schema;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,6 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.function.BiFunction;
import java.util.stream.Collectors;
import java.util.stream.Stream;

Expand Down Expand Up @@ -117,7 +116,7 @@ public PulsarDeserializer(SchemaInfo schemaInfo, JSONOptions parsedOptions, bool

case JSON:
FieldsDataType fdt = (FieldsDataType) rootDataType;
BiFunction<JsonFactory, String, JsonParser> createParser =
JacksonRecordParser.BiFunction<JsonFactory, String, JsonParser> createParser =
(jsonFactory, s) -> {
try {
return jsonFactory.createParser(s);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,76 @@ public void testStructTypesInAvro() throws Exception {
try {
see.execute("test struct in avro");
} catch (Exception e) {
}
SingletonStreamSink.compareWithList(
fooList.subList(0, fooList.size() - 1).stream().map(Objects::toString).collect(Collectors.toList()));
}

@Test
public void testStructTypesInJson() throws Exception {
StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment();
see.setParallelism(1);
StreamTableEnvironment tEnv = StreamTableEnvironment.create(see);

String table = newTopic() + "_json";

sendTypedMessages(table, SchemaType.JSON, fooList, Optional.empty(), SchemaData.Foo.class);

tEnv
.connect(getPulsarDescriptor(table))
.inAppendMode()
.registerTableSource(table);

Table t = tEnv.scan(table).select("i, f, bar");
tEnv.toAppendStream(t, t.getSchema().toRowType())
.map(new FailingIdentityMapper<Row>(fooList.size()))
.addSink(new SingletonStreamSink.StringSink<>()).setParallelism(1);

try {
see.execute("test struct in json");
} catch (Exception ignore) {
}
SingletonStreamSink.compareWithList(
fooList.subList(0, fooList.size() - 1).stream().map(Objects::toString).collect(Collectors.toList()));
}

@Test
public void testStructTypesInJsonBySql() throws Exception {
StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment();
see.setParallelism(1);
StreamTableEnvironment tEnv = StreamTableEnvironment.create(see);

String tableName = "topic_1_sql";
sendTypedMessages(tableName, SchemaType.JSON, fooList, Optional.empty(), SchemaData.Foo.class);

String createSql = "create table `" + tableName + "`(\n" +
"`b` BOOLEAN,\n" +
"`s` STRING\n" +
") with (\n" +
"'connector.type' ='pulsar',\n" +
"'connector.topic' ='persistent://public/default/" + tableName + "',\n" +
"'connector.service-url' ='" + serviceUrl + "',\n" +
"'connector.admin-url' ='" + adminUrl + "',\n" +
"'connector.startup-mode' ='earliest',\n" +
"'connector.properties.0.key' ='pulsar.reader.readerName',\n" +
"'connector.properties.0.value' ='testStructTypesInJsonBySql',\n" +
"'connector.properties.1.key' ='partitiondiscoveryintervalmillis',\n" +
"'connector.properties.1.value' = '5000',\n" +
"'format.derive-schema' ='true',\n" +
"'format.ignore-parse-errors' ='true',\n" +
"'update-mode' ='append'\n" +
")";
tEnv.sqlUpdate(createSql);

String querySql = "select i,f,bar from " + tableName;
Table table = tEnv.sqlQuery(querySql);
tEnv.toAppendStream(table, table.getSchema().toRowType())
.map(new FailingIdentityMapper<Row>(fooList.size()))
.addSink(new SingletonStreamSink.StringSink<>()).setParallelism(1);

try {
see.execute("test struct in json by sql");
} catch (Exception ignore) {
}
SingletonStreamSink.compareWithList(
fooList.subList(0, fooList.size() - 1).stream().map(Objects::toString).collect(Collectors.toList()));
Expand Down