Skip to content

Commit

Permalink
[FLINK-36311][format/json] Remove deprecated APIs in Flink Json format
Browse files Browse the repository at this point in the history
  • Loading branch information
dianfu committed Sep 18, 2024
1 parent d3950c5 commit a6895b6
Show file tree
Hide file tree
Showing 7 changed files with 13 additions and 120 deletions.

This file was deleted.

This file was deleted.

6 changes: 6 additions & 0 deletions flink-python/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,12 @@ under the License.
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-json</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>

<!-- Beam dependencies -->

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@

package org.apache.flink.formats.json;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
Expand Down Expand Up @@ -78,28 +78,23 @@
* <p>Deserializes a <code>byte[]</code> message as a JSON object and reads the specified fields.
*
* <p>Failures during deserialization are forwarded as wrapped IOExceptions.
*
* @deprecated The format was developed for the Table API users and will not be maintained for
* DataStream API users anymore. Either use Table API or switch to Data Stream, defining your
* own {@link DeserializationSchema}.
*/
@PublicEvolving
@Deprecated
@Internal
public class JsonRowDeserializationSchema implements DeserializationSchema<Row> {

private static final long serialVersionUID = -228294330688809195L;

/** Type information describing the result type. */
private final RowTypeInfo typeInfo;

private boolean failOnMissingField;
private final boolean failOnMissingField;

private final boolean hasDecimalType;

/** Object mapper for parsing the JSON. */
private transient ObjectMapper objectMapper;

private DeserializationRuntimeConverter runtimeConverter;
private final DeserializationRuntimeConverter runtimeConverter;

/** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */
private final boolean ignoreParseErrors;
Expand Down Expand Up @@ -128,26 +123,14 @@ public void open(InitializationContext context) throws Exception {
}
}

/** @deprecated Use the provided {@link Builder} instead. */
@Deprecated
public JsonRowDeserializationSchema(TypeInformation<Row> typeInfo) {
this(typeInfo, false, false);
}

/** @deprecated Use the provided {@link Builder} instead. */
@Deprecated
public JsonRowDeserializationSchema(String jsonSchema) {
this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema)), false, false);
}

/** @deprecated Use the provided {@link Builder} instead. */
@Deprecated
public void setFailOnMissingField(boolean failOnMissingField) {
// TODO make this class immutable once we drop this method
this.failOnMissingField = failOnMissingField;
this.runtimeConverter = createConverter(this.typeInfo);
}

@Override
public Row deserialize(byte[] message) throws IOException {
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.flink.formats.json;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.serialization.SerializationSchema;
import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
Expand Down Expand Up @@ -65,13 +65,8 @@
*
* <p>Result <code>byte[]</code> messages can be deserialized using {@link
* JsonRowDeserializationSchema}.
*
* @deprecated The format was developed for the Table API users and will not be maintained for
* DataStream API users anymore. Either use Table API or switch to Data Stream, defining your
* own {@link SerializationSchema}.
*/
@PublicEvolving
@Deprecated
@Internal
public class JsonRowSerializationSchema implements SerializationSchema<Row> {

private static final long serialVersionUID = -2885556750743978636L;
Expand Down Expand Up @@ -115,9 +110,7 @@ private Builder() {
*
* @param typeInfo Type information describing the result type. The field names of {@link
* Row} are used to parse the JSON properties.
* @deprecated Use {@link JsonRowSerializationSchema#builder()} instead.
*/
@Deprecated
public Builder(TypeInformation<Row> typeInfo) {
checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported");
this.typeInfo = (RowTypeInfo) typeInfo;
Expand All @@ -128,9 +121,7 @@ public Builder(TypeInformation<Row> typeInfo) {
*
* @param jsonSchema JSON schema describing the result type
* @see <a href="http://json-schema.org/">http://json-schema.org/</a>
* @deprecated Use {@link JsonRowSerializationSchema#builder()} instead.
*/
@Deprecated
public Builder(String jsonSchema) {
this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema)));
}
Expand Down Expand Up @@ -158,7 +149,7 @@ public JsonRowSerializationSchema build() {
}
}

/** Creates a builder for {@link JsonRowSerializationSchema.Builder}. */
/** Creates a builder for {@link Builder}. */
public static Builder builder() {
return new Builder();
}
Expand Down

0 comments on commit a6895b6

Please sign in to comment.