yunfengzhou-hub commented on code in PR #188: URL: https://github.com/apache/flink-ml/pull/188#discussion_r1057097049
########## flink-ml-core/src/test/java/org/apache/flink/ml/common/datastream/TableUtilsTest.java: ########## @@ -0,0 +1,75 @@ +/* + * 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.ml.common.datastream; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.types.DataType; +import org.apache.flink.types.Row; + +import org.apache.commons.lang3.ArrayUtils; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.sql.Timestamp; + +/** Tests the {@link TableUtils}. */ +public class TableUtilsTest { + private StreamExecutionEnvironment env; + private StreamTableEnvironment tEnv; + + @Before + public void before() { + env = StreamExecutionEnvironment.getExecutionEnvironment(); + tEnv = StreamTableEnvironment.create(env); + } + + @Test + public void testGetRowTypeInfo() { + Table inputTable = + tEnv.fromDataStream( + env.fromElements(new Timestamp(0)), Review Comment: Let's also add tests to verify the conversion of other data types as well. So far as I can see, we should include tests for data types listed [here](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/types/#list-of-data-types) and the vector/matrix data types introduced by Flink ML. ########## flink-ml-lib/src/main/java/org/apache/flink/ml/feature/binarizer/Binarizer.java: ########## @@ -70,11 +71,12 @@ public Table[] transform(Table... inputs) { for (int i = 0; i < inputCols.length; ++i) { int idx = inputTypeInfo.getFieldIndex(inputCols[i]); - if (inputTypeInfo.getFieldTypes()[idx] instanceof SparseVectorTypeInfo) { + Class<?> typeClass = inputTypeInfo.getTypeAt(idx).getTypeClass(); + if (typeClass.equals(SparseVector.class)) { outputTypes[i] = SparseVectorTypeInfo.INSTANCE; - } else if (inputTypeInfo.getFieldTypes()[idx] instanceof DenseVectorTypeInfo) { + } else if (typeClass.equals(DenseVector.class)) { Review Comment: We have implemented some performance optimization in the serialization process of dense vectors. If we cannot use these optimizations, Flink ML might experience obvious performance regression. Could you please evaluate the influence on performance If we use ExternalTypeInfo? -- 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. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
