Skip to content

Commit

Permalink
[FLINK-32008][protobuf] Ensure bulk persistence is not supported
Browse files Browse the repository at this point in the history
  • Loading branch information
RyanSkraba authored and libenchao committed Jun 17, 2023
1 parent f64563b commit 7d4ee28
Show file tree
Hide file tree
Showing 4 changed files with 126 additions and 0 deletions.
10 changes: 10 additions & 0 deletions flink-formats/flink-protobuf/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,16 @@ under the License.
<scope>provided</scope>
</dependency>

<!-- Filesystem connector is only present to ensure bulk files are not supported -->

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-files</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
<optional>true</optional>
</dependency>

<dependency>
<groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java</artifactId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
/*
* 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.formats.protobuf;

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.serialization.BulkWriter.Factory;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.connector.file.table.factories.BulkReaderFormatFactory;
import org.apache.flink.connector.file.table.factories.BulkWriterFormatFactory;
import org.apache.flink.connector.file.table.format.BulkDecodingFormat;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.connector.format.EncodingFormat;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.factories.DynamicTableFactory;

import java.util.Collections;
import java.util.Set;

/**
* Throw a {@link ValidationException} when using Protobuf format factory for file system.
*
* <p>In practice, there is <a href="https://protobuf.dev/programming-guides/techniques/#streaming">
* no standard</a> for storing bulk protobuf messages. This factory is present to prevent falling
* back to the {@link org.apache.flink.connector.file.table.DeserializationSchemaAdapter}, a
* line-based format which could silently succeed but write unrecoverable data to disk.
*
* <p>If your use case requires storing bulk protobuf messages on disk, the parquet file format
* might be the appropriate container and has an API for mapping records to protobuf messages.
*/
@Internal
public class PbFileFormatFactory implements BulkReaderFormatFactory, BulkWriterFormatFactory {

@Override
public String factoryIdentifier() {
return PbFormatFactory.IDENTIFIER;
}

@Override
public Set<ConfigOption<?>> requiredOptions() {
return Collections.emptySet();
}

@Override
public Set<ConfigOption<?>> optionalOptions() {
return Collections.emptySet();
}

@Override
public Set<ConfigOption<?>> forwardOptions() {
return Collections.emptySet();
}

@Override
public BulkDecodingFormat<RowData> createDecodingFormat(
DynamicTableFactory.Context context, ReadableConfig formatOptions) {
throw new ValidationException(
"The 'protobuf' format is not supported for the 'filesystem' connector.");
}

@Override
public EncodingFormat<Factory<RowData>> createEncodingFormat(
DynamicTableFactory.Context context, ReadableConfig formatOptions) {
throw new ValidationException(
"The 'protobuf' format is not supported for the 'filesystem' connector.");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -13,3 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.flink.formats.protobuf.PbFormatFactory
org.apache.flink.formats.protobuf.PbFileFormatFactory
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.formats.protobuf.testproto.MapTest;
import org.apache.flink.formats.protobuf.testproto.Pb3Test;
import org.apache.flink.table.api.TableResult;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.planner.runtime.utils.BatchTestBase;
import org.apache.flink.types.Row;
import org.apache.flink.util.CloseableIterator;
Expand All @@ -31,6 +32,8 @@
import java.util.Map;
import java.util.concurrent.ExecutionException;

import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
Expand Down Expand Up @@ -328,4 +331,33 @@ public void testSinkWithNullLiteralWithEscape() throws Exception {
MapTest.InnerMessageTest innerMessageTest = mapTest.getMap2Map().get("b");
assertEquals(MapTest.InnerMessageTest.getDefaultInstance(), innerMessageTest);
}

@Test
public void testUnsupportedBulkFilesystemSink() {
env().setParallelism(1);
String sql =
"create table bigdata_sink ( "
+ " a int, "
+ " map1 map<string,string>,"
+ " map2 map<string, row<a int, b bigint>>"
+ ") with ("
+ " 'connector' = 'filesystem', "
+ " 'path' = '/tmp/unused', "
+ " 'format' = 'protobuf', "
+ " 'protobuf.message-class-name' = 'org.apache.flink.formats.protobuf.testproto.MapTest'"
+ ")";
tEnv().executeSql(sql);

assertThatThrownBy(
() -> {
TableResult tableResult =
tEnv().executeSql(
"insert into bigdata_sink select 1, map['a', 'b', 'c', 'd'], map['f', row(1,cast(2 as bigint))] ");
tableResult.await();
})
.satisfies(
anyCauseMatches(
ValidationException.class,
"The 'protobuf' format is not supported for the 'filesystem' connector."));
}
}

0 comments on commit 7d4ee28

Please sign in to comment.