This is an automated email from the ASF dual-hosted git repository.

twalthr pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new ad47f6b  [FLINK-12253][table-common] Add a BINARY type
ad47f6b is described below

commit ad47f6bd409365102a500bf54096f0f2022824c7
Author: Timo Walther <twal...@apache.org>
AuthorDate: Tue Apr 30 16:43:41 2019 +0200

    [FLINK-12253][table-common] Add a BINARY type
---
 .../flink/table/types/logical/BinaryType.java      | 133 +++++++++++++++++++++
 .../table/types/logical/LogicalTypeVisitor.java    |   2 +
 .../apache/flink/table/types/LogicalTypesTest.java |  14 +++
 3 files changed, 149 insertions(+)

diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/BinaryType.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/BinaryType.java
new file mode 100644
index 0000000..2ae1af2
--- /dev/null
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/BinaryType.java
@@ -0,0 +1,133 @@
+/*
+ * 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.types.logical;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.api.ValidationException;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * Logical type of a fixed-length binary string (=a sequence of bytes).
+ *
+ * <p>The serialized string representation is {@code BINARY(n)} where {@code 
n} is the number of
+ * bytes. {@code n} must have a value between 1 and {@link Integer#MAX_VALUE} 
(both inclusive). If
+ * no length is specified, {@code n} is equal to 1.
+ */
+@PublicEvolving
+public final class BinaryType extends LogicalType {
+
+       private static final int MIN_LENGTH = 1;
+
+       private static final int MAX_LENGTH = Integer.MAX_VALUE;
+
+       private static final int DEFAULT_LENGTH = 1;
+
+       private static final String FORMAT = "BINARY(%d)";
+
+       private static final Set<String> INPUT_OUTPUT_CONVERSION = 
conversionSet(
+               byte[].class.getName(),
+               "org.apache.flink.table.dataformat.BinaryArray");
+
+       private static final Class<?> DEFAULT_CONVERSION = byte[].class;
+
+       private final int length;
+
+       public BinaryType(boolean isNullable, int length) {
+               super(isNullable, LogicalTypeRoot.BINARY);
+               if (length < MIN_LENGTH) {
+                       throw new ValidationException(
+                               String.format(
+                                       "Binary string length must be between 
%d and %d (both inclusive).",
+                                       MIN_LENGTH,
+                                       MAX_LENGTH));
+               }
+               this.length = length;
+       }
+
+       public BinaryType(int length) {
+               this(true, length);
+       }
+
+       public BinaryType() {
+               this(DEFAULT_LENGTH);
+       }
+
+       public int getLength() {
+               return length;
+       }
+
+       @Override
+       public LogicalType copy(boolean isNullable) {
+               return new BinaryType(isNullable, length);
+       }
+
+       @Override
+       public String asSerializableString() {
+               return withNullability(FORMAT, length);
+       }
+
+       @Override
+       public boolean supportsInputConversion(Class<?> clazz) {
+               return INPUT_OUTPUT_CONVERSION.contains(clazz.getName());
+       }
+
+       @Override
+       public boolean supportsOutputConversion(Class<?> clazz) {
+               return INPUT_OUTPUT_CONVERSION.contains(clazz.getName());
+       }
+
+       @Override
+       public Class<?> getDefaultConversion() {
+               return DEFAULT_CONVERSION;
+       }
+
+       @Override
+       public List<LogicalType> getChildren() {
+               return Collections.emptyList();
+       }
+
+       @Override
+       public <R> R accept(LogicalTypeVisitor<R> visitor) {
+               return visitor.visit(this);
+       }
+
+       @Override
+       public boolean equals(Object o) {
+               if (this == o) {
+                       return true;
+               }
+               if (o == null || getClass() != o.getClass()) {
+                       return false;
+               }
+               if (!super.equals(o)) {
+                       return false;
+               }
+               BinaryType that = (BinaryType) o;
+               return length == that.length;
+       }
+
+       @Override
+       public int hashCode() {
+               return Objects.hash(super.hashCode(), length);
+       }
+}
diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/LogicalTypeVisitor.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/LogicalTypeVisitor.java
index bbbc064..509b84b 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/LogicalTypeVisitor.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/LogicalTypeVisitor.java
@@ -35,5 +35,7 @@ public interface LogicalTypeVisitor<R> {
 
        R visit(BooleanType booleanType);
 
+       R visit(BinaryType binaryType);
+
        R visit(LogicalType other);
 }
diff --git 
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypesTest.java
 
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypesTest.java
index a8c903c..e764679 100644
--- 
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypesTest.java
+++ 
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypesTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.types;
 
+import org.apache.flink.table.types.logical.BinaryType;
 import org.apache.flink.table.types.logical.BooleanType;
 import org.apache.flink.table.types.logical.CharType;
 import org.apache.flink.table.types.logical.LogicalType;
@@ -78,6 +79,19 @@ public class LogicalTypesTest {
                );
        }
 
+       @Test
+       public void testBinaryType() {
+               testAll(
+                       new BinaryType(22),
+                       "BINARY(22)",
+                       "BINARY(22)",
+                       new Class[]{byte[].class},
+                       new Class[]{byte[].class},
+                       new LogicalType[]{},
+                       new BinaryType()
+               );
+       }
+
        // 
--------------------------------------------------------------------------------------------
 
        private static void testAll(

Reply via email to