Skip to content

Commit

Permalink
[FLINK-30093] Fixed outerProtoPrefix generation handling for enum
Browse files Browse the repository at this point in the history
  • Loading branch information
laughingman7743 committed Jan 8, 2023
1 parent 5fb41bb commit e86b868
Show file tree
Hide file tree
Showing 8 changed files with 154 additions and 32 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ public static String getFullJavaName(Descriptors.Descriptor descriptor) {
return parentJavaFullName + "." + descriptor.getName();
} else {
// top level message
String outerProtoName = getOuterProtoPrefix(descriptor);
String outerProtoName = getOuterProtoPrefix(descriptor.getFile());
return outerProtoName + descriptor.getName();
}
}
Expand All @@ -46,7 +46,8 @@ public static String getFullJavaName(Descriptors.EnumDescriptor enumDescriptor)
+ "."
+ enumDescriptor.getName();
} else {
return enumDescriptor.getFullName();
String outerProtoName = getOuterProtoPrefix(enumDescriptor.getFile());
return outerProtoName + enumDescriptor.getName();
}
}

Expand All @@ -71,32 +72,44 @@ public static String getStrongCamelCaseJsonName(String name) {
return ProtobufInternalUtils.underScoreToCamelCase(name, true);
}

public static String getOuterProtoPrefix(Descriptors.Descriptor descriptor) {
public static String getOuterClassName(Descriptors.FileDescriptor fileDescriptor) {
String[] fileNames = fileDescriptor.getName().split("/");
String fileName = fileNames[fileNames.length - 1];
String outerName = getStrongCamelCaseJsonName(fileName.split("\\.")[0]);
// https://developers.google.com/protocol-buffers/docs/reference/java-generated#invocation
// The name of the wrapper class is determined by converting the base name of the .proto
// file to camel case if the java_outer_classname option is not specified.
// For example, foo_bar.proto produces the class name FooBar. If there is a service,
// enum, or message (including nested types) in the file with the same name,
// "OuterClass" will be appended to the wrapper class's name.
boolean hasSameNameMessage =
fileDescriptor.getMessageTypes().stream()
.anyMatch(f -> f.getName().equals(outerName));
boolean hasSameNameEnum =
fileDescriptor.getEnumTypes().stream().anyMatch(f -> f.getName().equals(outerName));
boolean hasSameNameService =
fileDescriptor.getServices().stream().anyMatch(f -> f.getName().equals(outerName));
if (hasSameNameMessage || hasSameNameEnum || hasSameNameService) {
return outerName + PbConstant.PB_OUTER_CLASS_SUFFIX;
} else {
return outerName;
}
}

public static String getOuterProtoPrefix(Descriptors.FileDescriptor fileDescriptor) {
String javaPackageName =
descriptor.getFile().getOptions().hasJavaPackage()
? descriptor.getFile().getOptions().getJavaPackage()
: descriptor.getFile().getPackage();
if (descriptor.getFile().getOptions().getJavaMultipleFiles()) {
fileDescriptor.getOptions().hasJavaPackage()
? fileDescriptor.getOptions().getJavaPackage()
: fileDescriptor.getPackage();
if (fileDescriptor.getOptions().getJavaMultipleFiles()) {
return javaPackageName + ".";
}
if (descriptor.getFile().getOptions().hasJavaOuterClassname()) {
String outerName = descriptor.getFile().getOptions().getJavaOuterClassname();
return javaPackageName + "." + outerName + ".";
if (fileDescriptor.getOptions().hasJavaOuterClassname()) {
String outerClassName = fileDescriptor.getOptions().getJavaOuterClassname();
return javaPackageName + "." + outerClassName + ".";
} else {
String[] fileNames = descriptor.getFile().getName().split("/");
String fileName = fileNames[fileNames.length - 1];
String outerName = getStrongCamelCaseJsonName(fileName.split("\\.")[0]);
// https://developers.google.com/protocol-buffers/docs/reference/java-generated#invocation
// The name of the wrapper class is determined by converting the base name of the .proto
// file to camel case if the java_outer_classname option is not specified.
// For example, foo_bar.proto produces the class name FooBar. If there is a service,
// enum, or message (including nested types) in the file with the same name,
// "OuterClass" will be appended to the wrapper class's name.
if (outerName.equals(descriptor.getName())) {
return javaPackageName + "." + outerName + PbConstant.PB_OUTER_CLASS_SUFFIX + ".";
} else {
return javaPackageName + "." + outerName + ".";
}
String outerClassName = getOuterClassName(fileDescriptor);
return javaPackageName + "." + outerClassName + ".";
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,19 +26,36 @@
import static org.junit.Assert.assertEquals;

/** Test conversion of proto same outer class name data to flink internal data. */
public class SameOuterClassNameTest {
public class SameOuterClassNameProtoToRowTest {

@Test
public void testSimple() throws Exception {
TestSameOuterClassNameOuterClass.TestSameOuterClassName testSameOuterClassName =
TestSameOuterClassNameOuterClass.TestSameOuterClassName.newBuilder()
.setA(1)
.setB(TestSameOuterClassNameOuterClass.FooBar.BAR)
.build();
RowData row =
ProtobufTestHelper.pbBytesToRow(
TestSameOuterClassNameOuterClass.TestSameOuterClassName.class,
testSameOuterClassName.toByteArray());

assertEquals(1, row.getInt(0));
assertEquals("BAR", row.getString(1).toString());
}

@Test
public void testIntEnum() throws Exception {
TestSameOuterClassNameOuterClass.TestSameOuterClassName testSameOuterClassName =
TestSameOuterClassNameOuterClass.TestSameOuterClassName.newBuilder()
.setB(TestSameOuterClassNameOuterClass.FooBar.BAR)
.build();

RowData row =
ProtobufTestHelper.pbBytesToRow(
TestSameOuterClassNameOuterClass.TestSameOuterClassName.class,
testSameOuterClassName.toByteArray(),
true);
assertEquals(1, row.getInt(1));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* 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.formats.protobuf.testproto.TestSameOuterClassNameOuterClass;
import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.StringData;

import org.junit.Test;

import static org.junit.Assert.assertEquals;

/** Test conversion of flink internal primitive data to same outer class name proto data. */
public class SameOuterClassNameRowToProtoTest {
@Test
public void testSimple() throws Exception {
RowData row = GenericRowData.of(1, StringData.fromString("BAR"));

byte[] bytes =
ProtobufTestHelper.rowToPbBytes(
row, TestSameOuterClassNameOuterClass.TestSameOuterClassName.class);
TestSameOuterClassNameOuterClass.TestSameOuterClassName testSameOuterClassName =
TestSameOuterClassNameOuterClass.TestSameOuterClassName.parseFrom(bytes);
assertEquals(1, testSameOuterClassName.getA());
assertEquals(TestSameOuterClassNameOuterClass.FooBar.BAR, testSameOuterClassName.getB());
}

@Test
public void testEnumAsInt() throws Exception {
RowData row = GenericRowData.of(1, 1);

byte[] bytes =
ProtobufTestHelper.rowToPbBytes(
row, TestSameOuterClassNameOuterClass.TestSameOuterClassName.class, true);
TestSameOuterClassNameOuterClass.TestSameOuterClassName testSameOuterClassName =
TestSameOuterClassNameOuterClass.TestSameOuterClassName.parseFrom(bytes);
assertEquals(TestSameOuterClassNameOuterClass.FooBar.BAR, testSameOuterClassName.getB());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.flink.formats.protobuf;

import org.apache.flink.formats.protobuf.testproto.SimpleTestMulti;
import org.apache.flink.formats.protobuf.testproto.Status;
import org.apache.flink.table.data.RowData;

import com.google.protobuf.ByteString;
Expand All @@ -43,13 +44,14 @@ public void testSimple() throws Exception {
.setF("haha")
.setG(ByteString.copyFrom(new byte[] {1}))
.setH(SimpleTestMulti.Corpus.IMAGES)
.setI(Status.FINISHED)
.setFAbc7D(1) // test fieldNameToJsonName
.setVpr6S(2)
.build();

RowData row = ProtobufTestHelper.pbBytesToRow(SimpleTestMulti.class, simple.toByteArray());

assertEquals(10, row.getArity());
assertEquals(11, row.getArity());
assertEquals(1, row.getInt(0));
assertEquals(2L, row.getLong(1));
assertFalse(row.getBoolean(2));
Expand All @@ -58,8 +60,9 @@ public void testSimple() throws Exception {
assertEquals("haha", row.getString(5).toString());
assertEquals(1, (row.getBinary(6))[0]);
assertEquals("IMAGES", row.getString(7).toString());
assertEquals(1, row.getInt(8));
assertEquals(2, row.getInt(9));
assertEquals("FINISHED", row.getString(8).toString());
assertEquals(1, row.getInt(9));
assertEquals(2, row.getInt(10));
}

@Test
Expand Down Expand Up @@ -98,6 +101,7 @@ public void testDefaultValues() throws Exception {
assertFalse(row.isNullAt(5));
assertFalse(row.isNullAt(6));
assertFalse(row.isNullAt(7));
assertFalse(row.isNullAt(8));
assertEquals(10, row.getInt(0));
assertEquals(100L, row.getLong(1));
assertFalse(row.getBoolean(2));
Expand All @@ -106,14 +110,19 @@ public void testDefaultValues() throws Exception {
assertEquals("f", row.getString(5).toString());
assertArrayEquals(ByteString.EMPTY.toByteArray(), row.getBinary(6));
assertEquals(SimpleTestMulti.Corpus.UNIVERSAL.toString(), row.getString(7).toString());
assertEquals(Status.UNSPECIFIED.toString(), row.getString(8).toString());
}

@Test
public void testIntEnum() throws Exception {
SimpleTestMulti simple =
SimpleTestMulti.newBuilder().setH(SimpleTestMulti.Corpus.IMAGES).build();
SimpleTestMulti.newBuilder()
.setH(SimpleTestMulti.Corpus.IMAGES)
.setI(Status.STARTED)
.build();
RowData row =
ProtobufTestHelper.pbBytesToRow(SimpleTestMulti.class, simple.toByteArray(), true);
assertEquals(2, row.getInt(7));
assertEquals(1, row.getInt(8));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.flink.formats.protobuf;

import org.apache.flink.formats.protobuf.testproto.SimpleTestMulti;
import org.apache.flink.formats.protobuf.testproto.Status;
import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.StringData;
Expand All @@ -43,6 +44,7 @@ public void testSimple() throws Exception {
StringData.fromString("hello"),
new byte[] {1},
StringData.fromString("IMAGES"),
StringData.fromString("FINISHED"),
1,
2);

Expand All @@ -57,6 +59,7 @@ public void testSimple() throws Exception {
assertEquals("hello", simpleTestMulti.getF());
assertEquals(1, simpleTestMulti.getG().byteAt(0));
assertEquals(SimpleTestMulti.Corpus.IMAGES, simpleTestMulti.getH());
assertEquals(Status.FINISHED, simpleTestMulti.getI());
assertEquals(1, simpleTestMulti.getFAbc7D());
}

Expand All @@ -72,6 +75,7 @@ public void testNull() throws Exception {
StringData.fromString("hello"),
null,
null,
null,
1,
2);

Expand All @@ -80,17 +84,20 @@ public void testNull() throws Exception {
assertFalse(simpleTestMulti.hasA());
assertFalse(simpleTestMulti.hasG());
assertFalse(simpleTestMulti.hasH());
assertFalse(simpleTestMulti.hasI());
}

@Test
public void testEnumAsInt() throws Exception {
RowData row =
GenericRowData.of(
null, null, null, null, null, null, null, 2, // CORPUS: IMAGE
1, // STATUS: STARTED
null, null);

byte[] bytes = ProtobufTestHelper.rowToPbBytes(row, SimpleTestMulti.class, true);
SimpleTestMulti simpleTestMulti = SimpleTestMulti.parseFrom(bytes);
assertEquals(SimpleTestMulti.Corpus.IMAGES, simpleTestMulti.getH());
assertEquals(Status.STARTED, simpleTestMulti.getI());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,4 +22,10 @@ option java_package = "org.apache.flink.formats.protobuf.testproto";

message TestSameOuterClassName {
int32 a = 1;
FooBar b = 2;
}

enum FooBar {
FOO = 0;
BAR = 1;
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,12 +27,13 @@ message SimpleTestMulti {
optional bool c = 3;
optional float d = 4;
optional double e = 5;
optional string f = 6 [default = "f"];;
optional string f = 6 [default = "f"];
optional bytes g = 7;
optional Corpus h = 8;
optional Status i = 9;
//this is must because protobuf have some field name parse bug if number is after "_".
optional int32 f_abc_7d = 9;
optional int32 vpr6s = 10;
optional int32 f_abc_7d = 10;
optional int32 vpr6s = 11;

enum Corpus {
UNIVERSAL = 0;
Expand All @@ -44,3 +45,10 @@ message SimpleTestMulti {
VIDEO = 7;
}
}

enum Status {
UNSPECIFIED = 0;
STARTED = 1;
RUNNING = 2;
FINISHED = 3;
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,15 @@ message SimpleTestNoMulti {
optional string f = 6;
optional bytes g = 7;
optional InnerMessageTest h = 8;
optional Result i = 9;

message InnerMessageTest{
optional int32 a = 1;
optional int64 b = 2;
}
}

enum Result {
SUCCESS = 0;
FAIL = 1;
}

0 comments on commit e86b868

Please sign in to comment.