gyfora commented on a change in pull request #78: Kudu Connector rework URL: https://github.com/apache/bahir-flink/pull/78#discussion_r410415941
########## File path: flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/table/utils/KuduTableUtils.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.connectors.kudu.table.utils; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connectors.kudu.connector.ColumnSchemasFactory; +import org.apache.flink.connectors.kudu.connector.CreateTableOptionsFactory; +import org.apache.flink.connectors.kudu.connector.KuduTableInfo; +import org.apache.flink.connectors.kudu.table.KuduTableFactory; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.utils.TableSchemaUtils; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + +import org.apache.kudu.ColumnSchema; +import org.apache.kudu.ColumnTypeAttributes; +import org.apache.kudu.Schema; +import org.apache.kudu.client.CreateTableOptions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.flink.connectors.kudu.table.KuduTableFactory.KUDU_HASH_COLS; +import static org.apache.flink.connectors.kudu.table.KuduTableFactory.KUDU_PRIMARY_KEY_COLS; + +public class KuduTableUtils { + + private static final Logger LOG = LoggerFactory.getLogger(KuduTableUtils.class); + + public static KuduTableInfo createTableInfo(String tableName, TableSchema schema, Map<String, String> props) { + + boolean createIfMissing = props.containsKey(KUDU_HASH_COLS); + List<Tuple2<String, DataType>> columns = TableSchemaUtils + .getPhysicalSchema(schema) + .getTableColumns() + .stream() + .map(tc -> Tuple2.of(tc.getName(), tc.getType())) + .collect(Collectors.toList()); + + KuduTableInfo tableInfo = KuduTableInfo.forTable(tableName); + + if (createIfMissing) { + List<String> keyColumns = getPrimaryKeyColumns(props, schema); + ColumnSchemasFactory schemasFactory = () -> toKuduConnectorColumns(columns, keyColumns); + List<String> hashColumns = getHashColumns(props); + int replicas = Optional.ofNullable(props.get(KuduTableFactory.KUDU_REPLICAS)).map(Integer::parseInt).orElse(1); + + CreateTableOptionsFactory optionsFactory = () -> new CreateTableOptions() + .setNumReplicas(replicas) + .addHashPartitions(hashColumns, replicas * 2); + + tableInfo.createTableIfNotExists(schemasFactory, optionsFactory); + } else { + LOG.debug("Property {} is missing, assuming the table is already created.", KUDU_HASH_COLS); + } + + return tableInfo; + } + + public static List<ColumnSchema> toKuduConnectorColumns(List<Tuple2<String, DataType>> columns, Collection<String> keyColumns) { + return columns.stream() + .map(t -> { + ColumnSchema.ColumnSchemaBuilder builder = new ColumnSchema + .ColumnSchemaBuilder(t.f0, KuduTypeUtils.toKuduType(t.f1)) + .key(keyColumns.contains(t.f0)) + .nullable(!keyColumns.contains(t.f0) && t.f1.getLogicalType().isNullable()); Review comment: The Flink/Calcite type checking for relational queries takes nullability a bit too seriously at this point for type checking. It is almost impossible to use NOT Null types in queries or with other connectors. Due to this limitation we decided to not treat Kudu key columns as non-nullable on the Flink side. What this means is that we dont require kudu key columns to be NOT NULL in Flink and rely only for the key column property for making it a Kudu key column (which we have to set non nullable in the kudu schema) ---------------------------------------------------------------- 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: [email protected] With regards, Apache Git Services
