You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/07/22 02:26:25 UTC

[GitHub] [flink] fsk119 commented on a change in pull request #11896: [FLINK-14356] [single-value] Introduce "single-value" format to (de)serialize message to a single field

fsk119 commented on a change in pull request #11896:
URL: https://github.com/apache/flink/pull/11896#discussion_r458104873



##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/format/single/SingleValue.java
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.table.format.single;
+
+import org.apache.flink.table.descriptors.DescriptorProperties;
+import org.apache.flink.table.descriptors.FormatDescriptor;
+
+import java.util.Map;
+
+import static org.apache.flink.table.format.single.SingleValueValidator.FORMAT_TYPE_VALUE;

Review comment:
       Please add test for added class.

##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/format/single/SingleValueRowSerializer.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.table.format.single;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.base.BooleanSerializer;
+import org.apache.flink.api.common.typeutils.base.ByteSerializer;
+import org.apache.flink.api.common.typeutils.base.DoubleSerializer;
+import org.apache.flink.api.common.typeutils.base.FloatSerializer;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.ShortSerializer;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.RowData.FieldGetter;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.io.IOException;
+
+/**
+ * SingleValueSerializer: deserialize or serialize single value.
+ */
+public class SingleValueRowSerializer implements DeserializationSchema<RowData>,
+	SerializationSchema<RowData> {

Review comment:
       Class name is not suitable if you want to implement two interface together.

##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/format/single/SingleValueValidator.java
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.table.format.single;
+
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+import org.apache.flink.table.descriptors.FormatDescriptorValidator;
+import org.apache.flink.table.factories.TableFormatFactoryBase;
+
+/**
+ * Single value format validator.
+ */
+
+public class SingleValueValidator extends FormatDescriptorValidator{
+
+	public static final String FORMAT_TYPE_VALUE = "single-value";
+
+	@Override
+	public void validate(DescriptorProperties properties) {
+		super.validate(properties);
+
+		TableSchema tableSchema = TableFormatFactoryBase.deriveSchema(properties.asMap());
+
+		if (tableSchema.getFieldCount() > 1) {

Review comment:
       Validator works only for Descriptor api. Currently community is prepare to refactor Descriptor api. I think you can leave it as later work.

##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/format/single/SingleValueRowSerializer.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.table.format.single;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.base.BooleanSerializer;
+import org.apache.flink.api.common.typeutils.base.ByteSerializer;
+import org.apache.flink.api.common.typeutils.base.DoubleSerializer;
+import org.apache.flink.api.common.typeutils.base.FloatSerializer;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.ShortSerializer;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.RowData.FieldGetter;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.io.IOException;
+
+/**
+ * SingleValueSerializer: deserialize or serialize single value.
+ */
+public class SingleValueRowSerializer implements DeserializationSchema<RowData>,
+	SerializationSchema<RowData> {
+
+	private TypeInformation<RowData> typeInfo;
+	private LogicalType singleValueLogicalType;
+	private SingleValueSerializer singleValueSerializer;
+	private FieldGetter fieldGetter;
+
+	public SingleValueRowSerializer(RowType rowType,
+		TypeInformation<RowData> resultTypeInfo) {
+		this.typeInfo = resultTypeInfo;
+		this.singleValueLogicalType = rowType.getTypeAt(0);
+		this.fieldGetter = RowData.createFieldGetter(singleValueLogicalType, 0);
+		this.singleValueSerializer = initSingleValueSerializer(singleValueLogicalType.getTypeRoot());
+	}
+
+	@Override
+	public RowData deserialize(byte[] message) throws IOException {
+		GenericRowData genericRowData = new GenericRowData(1);
+		genericRowData.setField(0, singleValueSerializer.deserialize(message));
+		return genericRowData;
+	}
+
+	@Override
+	public boolean isEndOfStream(RowData nextElement) {
+		return false;
+	}
+
+	@Override
+	public TypeInformation<RowData> getProducedType() {
+		return typeInfo;
+	}
+
+	@Override
+	public byte[] serialize(RowData element) {
+		return singleValueSerializer.serialize(fieldGetter.getFieldOrNull(element));
+	}
+
+	private SingleValueSerializer initSingleValueSerializer(LogicalTypeRoot typeRoot) {
+		switch (typeRoot) {
+			case CHAR:
+			case VARCHAR:
+				return new StringSingleValueSerializer();
+			case VARBINARY:
+			case BINARY:
+				return new BytesSingleValueSerializer();
+			case TINYINT:
+				return new BasicSingleValueSerializer(ByteSerializer.INSTANCE, BasicTypeInfo.BYTE_TYPE_INFO);
+			case SMALLINT:
+				return new BasicSingleValueSerializer(ShortSerializer.INSTANCE, BasicTypeInfo.SHORT_TYPE_INFO);
+			case INTEGER:
+				return new BasicSingleValueSerializer(IntSerializer.INSTANCE, BasicTypeInfo.INT_TYPE_INFO);
+			case BIGINT:
+				return new BasicSingleValueSerializer(LongSerializer.INSTANCE, BasicTypeInfo.LONG_TYPE_INFO);
+			case FLOAT:
+				return new BasicSingleValueSerializer(FloatSerializer.INSTANCE, BasicTypeInfo.FLOAT_TYPE_INFO);
+			case DOUBLE:
+				return new BasicSingleValueSerializer(DoubleSerializer.INSTANCE, BasicTypeInfo.DOUBLE_TYPE_INFO);
+			case BOOLEAN:
+				return new BasicSingleValueSerializer(BooleanSerializer.INSTANCE, BasicTypeInfo.BOOLEAN_TYPE_INFO);
+			default:
+				throw new RuntimeException("Unsupported single value type:" + singleValueLogicalType.getTypeRoot());
+		}
+	}
+
+	@Override
+	public boolean equals(Object o) {

Review comment:
       Please compare all fields.

##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/format/single/SingleValueFormatFactory.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.table.format.single;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DeserializationFormatFactory;
+import org.apache.flink.table.factories.DynamicTableFactory.Context;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.SerializationFormatFactory;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.Collections;
+import java.util.Set;
+
+/**
+ * SingleValueFormatFactory for single value.
+ */
+public class SingleValueFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory {

Review comment:
       Please add @Interanl annotation for all interal public class.

##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/format/single/SingleValueRowSerializer.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.table.format.single;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.base.BooleanSerializer;
+import org.apache.flink.api.common.typeutils.base.ByteSerializer;
+import org.apache.flink.api.common.typeutils.base.DoubleSerializer;
+import org.apache.flink.api.common.typeutils.base.FloatSerializer;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.ShortSerializer;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.RowData.FieldGetter;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.io.IOException;
+
+/**
+ * SingleValueSerializer: deserialize or serialize single value.
+ */
+public class SingleValueRowSerializer implements DeserializationSchema<RowData>,
+	SerializationSchema<RowData> {
+
+	private TypeInformation<RowData> typeInfo;
+	private LogicalType singleValueLogicalType;
+	private SingleValueSerializer singleValueSerializer;
+	private FieldGetter fieldGetter;
+
+	public SingleValueRowSerializer(RowType rowType,
+		TypeInformation<RowData> resultTypeInfo) {
+		this.typeInfo = resultTypeInfo;
+		this.singleValueLogicalType = rowType.getTypeAt(0);
+		this.fieldGetter = RowData.createFieldGetter(singleValueLogicalType, 0);
+		this.singleValueSerializer = initSingleValueSerializer(singleValueLogicalType.getTypeRoot());
+	}
+
+	@Override
+	public RowData deserialize(byte[] message) throws IOException {
+		GenericRowData genericRowData = new GenericRowData(1);
+		genericRowData.setField(0, singleValueSerializer.deserialize(message));
+		return genericRowData;
+	}
+
+	@Override
+	public boolean isEndOfStream(RowData nextElement) {
+		return false;
+	}
+
+	@Override
+	public TypeInformation<RowData> getProducedType() {
+		return typeInfo;
+	}
+
+	@Override
+	public byte[] serialize(RowData element) {
+		return singleValueSerializer.serialize(fieldGetter.getFieldOrNull(element));
+	}
+
+	private SingleValueSerializer initSingleValueSerializer(LogicalTypeRoot typeRoot) {
+		switch (typeRoot) {
+			case CHAR:
+			case VARCHAR:

Review comment:
       Considering this serializer only works here, I think it's not suitable to add new public class here, such as `BasicSingleValueSerializer`.  
   You can add a private function interface such as `SerializationRuntimeConverter` and expose `serialize` method. Therefore, you can use lambda function here instead of adding new class. 
   You can take a look at `JsonRowDataSerializationSchema` .

##########
File path: flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/format/single/SingleValueFormatFactoryTest.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.table.format.single;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TestDynamicTableFactory;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link SingleValueFormatFactory}.
+ */
+public class SingleValueFormatFactoryTest extends TestLogger {
+	private static final TableSchema SCHEMA = TableSchema.builder()
+		.field("field1", DataTypes.STRING())
+		.build();
+
+	private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType();
+
+	@Test
+	public void testSeDeSchema() {

Review comment:
       Please compare actual sink and expected sink.

##########
File path: flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/serializers/python/StringSingleValueSerializerTest.java
##########
@@ -24,7 +24,7 @@
 /**
  * Test for {@link StringSerializer}.
  */
-public class StringSerializerTest extends SerializerTestBase<String> {
+public class StringSingleValueSerializerTest extends SerializerTestBase<String> {

Review comment:
       Why modify here? 
   

##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/format/single/SingleValueValidator.java
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.table.format.single;
+
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+import org.apache.flink.table.descriptors.FormatDescriptorValidator;
+import org.apache.flink.table.factories.TableFormatFactoryBase;
+
+/**
+ * Single value format validator.
+ */
+
+public class SingleValueValidator extends FormatDescriptorValidator{
+
+	public static final String FORMAT_TYPE_VALUE = "single-value";
+
+	@Override
+	public void validate(DescriptorProperties properties) {
+		super.validate(properties);
+
+		TableSchema tableSchema = TableFormatFactoryBase.deriveSchema(properties.asMap());
+
+		if (tableSchema.getFieldCount() > 1) {

Review comment:
       I think you can validte schema in SingleValueFormatFactory.

##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/format/single/SingleValueFormatFactory.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.table.format.single;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DeserializationFormatFactory;
+import org.apache.flink.table.factories.DynamicTableFactory.Context;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.SerializationFormatFactory;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.Collections;
+import java.util.Set;
+
+/**
+ * SingleValueFormatFactory for single value.
+ */
+public class SingleValueFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory {
+
+	public static final String IDENTIFIER = "single-value";

Review comment:
       String doesn't need final.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org