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-37154: Adding SimpleByteSchema for PyFlink #26080

Merged
merged 7 commits into from
Feb 7, 2025
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.
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
@@ -0,0 +1,54 @@
/*
* 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.
*/

package org.apache.flink.api.common.serialization;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.api.common.typeinfo.TypeInformation;

import java.io.IOException;

/** Simple serialization schema for bytes. */
@PublicEvolving
public class ByteArraySchema implements DeserializationSchema<byte[]>, SerializationSchema<byte[]> {

private static final long serialVersionUID = 1L;

// ------------------------------------------------------------------------
// Kafka Serialization
// ------------------------------------------------------------------------

@Override
public byte[] deserialize(byte[] message) throws IOException {
return message;
}

@Override
public boolean isEndOfStream(byte[] nextElement) {
return false;
}

@Override
public byte[] serialize(byte[] element) {
return element;
}

@Override
public TypeInformation<byte[]> getProducedType() {
return TypeInformation.of(byte[].class);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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.
*/

package org.apache.flink.api.common.serialization;

import org.junit.jupiter.api.Test;

import java.io.IOException;

import static org.assertj.core.api.Assertions.assertThat;

public class ByteArraySchemaTest {
@Test
void testSimpleSerialisation() throws IOException {
final byte[] testBytes = "hello world".getBytes();
assertThat(new ByteArraySchema().serialize(testBytes)).isEqualTo(testBytes);
assertThat(new ByteArraySchema().deserialize(testBytes)).isEqualTo(testBytes);
}
}
10 changes: 6 additions & 4 deletions flink-python/pyflink/common/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -45,8 +45,9 @@
:class:`~pyflink.datastream.formats.csv.CsvRowSerializationSchema`,
:class:`~pyflink.datastream.formats.csv.CsvRowDeserializationSchema`,
:class:`~pyflink.datastream.formats.avro.AvroRowSerializationSchema`,
:class:`~pyflink.datastream.formats.avro.AvroRowDeserializationSchema` and
:class:`~SimpleStringSchema` for more details.
:class:`~pyflink.datastream.formats.avro.AvroRowDeserializationSchema`,
:class:`~SimpleStringSchema` and
:class:`~SimpleByteSchema` for more details.
"""
from pyflink.common.completable_future import CompletableFuture
from pyflink.common.config_options import ConfigOption, ConfigOptions
Expand All @@ -59,11 +60,11 @@
from pyflink.common.job_status import JobStatus
from pyflink.common.restart_strategy import RestartStrategies, RestartStrategyConfiguration
from pyflink.common.serialization import SerializationSchema, DeserializationSchema, \
SimpleStringSchema, Encoder
SimpleStringSchema, ByteArraySchema, Encoder
from pyflink.common.serializer import TypeSerializer
from pyflink.common.time import Duration, Instant, Time
from pyflink.common.typeinfo import Types, TypeInformation
from pyflink.common.types import Row, RowKind
from pyflink.common.time import Duration, Instant, Time
from pyflink.common.watermark_strategy import WatermarkStrategy, \
AssignerWithPeriodicWatermarksWrapper

Expand All @@ -78,6 +79,7 @@
'SerializationSchema',
'DeserializationSchema',
'SimpleStringSchema',
'ByteArraySchema',
'Encoder',
'CompletableFuture',
'InputDependencyConstraint',
Expand Down
16 changes: 16 additions & 0 deletions flink-python/pyflink/common/serialization.py
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
'SerializationSchema',
'DeserializationSchema',
'SimpleStringSchema',
'ByteArraySchema',
'Encoder',
'BulkWriterFactory'
]
Expand Down Expand Up @@ -68,6 +69,21 @@ def __init__(self, charset: str = 'UTF-8'):
self, j_deserialization_schema=j_simple_string_serialization_schema)


class ByteArraySchema(SerializationSchema, DeserializationSchema):
"""
Simple serialization/deserialization schema for bytes.
"""

def __init__(self):
gate_way = get_gateway()
j_simple_byte_serialization_schema = gate_way \
.jvm.org.apache.flink.api.common.serialization.ByteArraySchema()
SerializationSchema.__init__(self,
j_serialization_schema=j_simple_byte_serialization_schema)
DeserializationSchema.__init__(
self, j_deserialization_schema=j_simple_byte_serialization_schema)


class Encoder(object):
"""
Encoder is used by the file sink to perform the actual writing of the
Expand Down
12 changes: 11 additions & 1 deletion flink-python/pyflink/common/tests/test_serialization_schemas.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
# See the License for the specific language governing permissions and
# limitations under the License.
################################################################################
from pyflink.common.serialization import SimpleStringSchema
from pyflink.common.serialization import SimpleStringSchema, ByteArraySchema
from pyflink.testing.test_case_utils import PyFlinkTestCase


Expand All @@ -29,3 +29,13 @@ def test_simple_string_schema(self):

self.assertEqual(expected_string, simple_string_schema._j_deserialization_schema
.deserialize(expected_string.encode(encoding='utf-8')))


class SimpleByteSchemaTests(PyFlinkTestCase):
def test_simple_byte_schema(self):
expected_bytes = "test bytes".encode(encoding='utf-8')
simple_byte_schema = ByteArraySchema()
self.assertEqual(expected_bytes,
simple_byte_schema._j_serialization_schema.serialize(expected_bytes))
self.assertEqual(expected_bytes, simple_byte_schema._j_deserialization_schema
.deserialize(expected_bytes))
10 changes: 7 additions & 3 deletions flink-python/pyflink/datastream/connectors/tests/test_kafka.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,14 +16,13 @@
# limitations under the License.
################################################################################
import json
from typing import Dict
import unittest
from typing import Dict

import pyflink.datastream.data_stream as data_stream
from pyflink.common import typeinfo

from pyflink.common.configuration import Configuration
from pyflink.common.serialization import SimpleStringSchema, DeserializationSchema
from pyflink.common.serialization import SimpleStringSchema, ByteArraySchema, DeserializationSchema
from pyflink.common.typeinfo import Types
from pyflink.common.types import Row
from pyflink.common.watermark_strategy import WatermarkStrategy
Expand Down Expand Up @@ -318,6 +317,7 @@ def _check(schema: DeserializationSchema, class_name: str):
class_name)

_check(SimpleStringSchema(), 'org.apache.flink.api.common.serialization.SimpleStringSchema')
_check(ByteArraySchema(), 'org.apache.flink.api.common.serialization.ByteArraySchema')
_check(
JsonRowDeserializationSchema.builder().type_info(Types.ROW([Types.STRING()])).build(),
'org.apache.flink.formats.json.JsonRowDeserializationSchema'
Expand Down Expand Up @@ -635,6 +635,10 @@ def _check_serialization_schema_implementations(check_function):
SimpleStringSchema(),
'org.apache.flink.api.common.serialization.SimpleStringSchema'
)
check_function(
ByteArraySchema(),
'org.apache.flink.api.common.serialization.ByteArraySchema'
)


class MockDataStream(data_stream.DataStream):
Expand Down