tsreaper commented on code in PR #121:
URL: https://github.com/apache/flink-table-store/pull/121#discussion_r876537585


##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/FileStoreImpl.java:
##########
@@ -201,17 +204,28 @@ public static FileStoreImpl createWithPrimaryKey(
                                 .collect(Collectors.toList()));
 
         MergeFunction mergeFunction;
+        Map<String, String> rightConfMap =
+                options.getFilterConf(e -> 
e.getKey().endsWith(".aggregate-function"));

Review Comment:
   Move this to `AGGREGATION` branch.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/AggregateFunction.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.store.file.mergetree.compact;
+
+import java.io.Serializable;
+
+/**
+ * 自定义的列聚合抽象类.

Review Comment:
   Flink Table Store is currently a sub-project of Flink. English comments only.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/AggregateFunction.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.store.file.mergetree.compact;
+
+import java.io.Serializable;
+
+/**
+ * 自定义的列聚合抽象类.
+ *
+ * @param <T>
+ */
+public interface AggregateFunction<T> extends Serializable {
+    //     T aggregator;
+
+    T getResult();
+
+    default void init() {
+        reset();
+    }
+
+    void reset();
+
+    default void aggregate(Object value) {
+        aggregate(value, true);
+    }
+
+    void aggregate(Object value, boolean add);
+
+    void reset(Object value);

Review Comment:
   What's the usage of this method?



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/AggregateFunction.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.store.file.mergetree.compact;
+
+import java.io.Serializable;
+
+/**
+ * 自定义的列聚合抽象类.
+ *
+ * @param <T>
+ */
+public interface AggregateFunction<T> extends Serializable {
+    //     T aggregator;

Review Comment:
   Remove useless code.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/AggregationMergeFunction.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.store.file.mergetree.compact;
+
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A {@link MergeFunction} where key is primary key (unique) and value is the 
partial record, update
+ * non-null fields on merge.
+ */
+@SuppressWarnings("checkstyle:RegexpSingleline")
+public class AggregationMergeFunction implements MergeFunction {
+
+    private static final long serialVersionUID = 1L;
+
+    private final RowData.FieldGetter[] getters;
+
+    private final RowType rowType;
+    private final ArrayList<AggregateFunction<?>> aggregateFunctions;
+    private final boolean[] isPrimaryKey;
+    private final RowType primaryKeyType;
+    private transient GenericRowData row;
+
+    private final Set<String> aggregateColumnNames;
+
+    public AggregationMergeFunction(
+            RowType primaryKeyType, RowType rowType, Set<String> 
aggregateColumnNames) {
+        this.primaryKeyType = primaryKeyType;
+        this.rowType = rowType;
+        this.aggregateColumnNames = aggregateColumnNames;
+
+        List<LogicalType> fieldTypes = rowType.getChildren();
+        this.getters = new RowData.FieldGetter[fieldTypes.size()];
+        for (int i = 0; i < fieldTypes.size(); i++) {
+            getters[i] = RowData.createFieldGetter(fieldTypes.get(i), i);
+        }
+
+        this.isPrimaryKey = new boolean[this.getters.length];
+        Arrays.fill(isPrimaryKey, false);
+        List<String> rowNames = rowType.getFieldNames();
+        for (String primaryKeyName : primaryKeyType.getFieldNames()) {
+            isPrimaryKey[rowNames.indexOf(primaryKeyName)] = true;
+        }
+
+        this.aggregateFunctions = new ArrayList<>(rowType.getFieldCount());
+        for (int i = 0; i < rowType.getFieldCount(); i++) {
+            AggregateFunction<?> f = null;
+            if (aggregateColumnNames.contains(rowNames.get(i))) {
+                f = 
choiceRightAggregateFunction(rowType.getTypeAt(i).getDefaultConversion());
+            }
+            aggregateFunctions.add(f);
+        }
+    }
+
+    private AggregateFunction<?> choiceRightAggregateFunction(Class<?> c) {
+        AggregateFunction<?> f = null;
+        if (Double.class.equals(c)) {
+            f = new DoubleAggregateFunction();
+        } else if (Long.class.equals(c)) {
+            f = new LongAggregateFunction();
+        } else if (Integer.class.equals(c)) {
+            f = new IntegerAggregateFunction();
+        } else if (Float.class.equals(c)) {
+            f = new FloatAggregateFunction();
+        }
+        return f;
+    }
+
+    @Override
+    public void reset() {
+        this.row = new GenericRowData(getters.length);
+    }
+
+    @Override
+    public void add(RowData value) {
+        for (int i = 0; i < getters.length; i++) {
+            Object currentField = getters[i].getFieldOrNull(value);
+            AggregateFunction<?> f = aggregateFunctions.get(i);
+            if (isPrimaryKey[i]) {
+                // primary key
+                if (currentField != null) {
+                    row.setField(i, currentField);
+                }
+            } else {
+                if (f != null) {
+                    f.reset();
+                    Object oldValue = row.getField(i);
+                    if (oldValue != null) {
+                        f.aggregate(oldValue);
+                    }
+                    switch (row.getRowKind()) {
+                        case INSERT:
+                            f.aggregate(currentField);
+                            break;
+                        case DELETE:
+                            f.aggregate(currentField, false);
+                            break;
+                        case UPDATE_AFTER:
+                        case UPDATE_BEFORE:
+                        default:
+                            throw new UnsupportedOperationException(
+                                    "Unsupported row kind: " + 
row.getRowKind());

Review Comment:
   You can treat `UPDATE_BEFORE` as `DELETE` and `UPDATE_AFTER` as `INSERT`. 
They're actually different in some other ways but in our scenario they're the 
same.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/FileStoreImpl.java:
##########
@@ -201,17 +204,28 @@ public static FileStoreImpl createWithPrimaryKey(
                                 .collect(Collectors.toList()));
 
         MergeFunction mergeFunction;
+        Map<String, String> rightConfMap =
+                options.getFilterConf(e -> 
e.getKey().endsWith(".aggregate-function"));
         switch (mergeEngine) {
             case DEDUPLICATE:
                 mergeFunction = new DeduplicateMergeFunction();
                 break;
             case PARTIAL_UPDATE:
-                List<LogicalType> fieldTypes = rowType.getChildren();
-                RowData.FieldGetter[] fieldGetters = new 
RowData.FieldGetter[fieldTypes.size()];
-                for (int i = 0; i < fieldTypes.size(); i++) {
-                    fieldGetters[i] = 
RowData.createFieldGetter(fieldTypes.get(i), i);
+                mergeFunction = new PartialUpdateMergeFunction(rowType);
+                break;
+            case AGGREGATION:
+                Set<String> valueSet = new HashSet<>(rightConfMap.values());
+                if (valueSet.size() != 1 || !valueSet.contains("sum")) {
+                    throw new IllegalArgumentException(
+                            "Aggregate function must be the same for all 
columns");

Review Comment:
   Use a factory class to produce corresponding aggregate function. If we'd 
like to support other aggregate functions in the future this implementation is 
not friendly for changes.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/AggregateFunction.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.store.file.mergetree.compact;
+
+import java.io.Serializable;
+
+/**
+ * 自定义的列聚合抽象类.
+ *
+ * @param <T>

Review Comment:
   Explain what `T` is. You can also remove this line if there is nothing to 
explain.



##########
flink-table-store-connector/src/test/java/org/apache/flink/table/store/connector/AggregationITCase.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.store.connector;
+
+import org.apache.flink.types.Row;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.flink.util.CollectionUtil.iteratorToList;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** ITCase for partial update. */
+public class AggregationITCase extends FileStoreTableITCase {

Review Comment:
   There are no tests for retraction. Let me find you an example.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/AggregateFunction.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.store.file.mergetree.compact;
+
+import java.io.Serializable;
+
+/**
+ * 自定义的列聚合抽象类.
+ *
+ * @param <T>
+ */
+public interface AggregateFunction<T> extends Serializable {
+    //     T aggregator;
+
+    T getResult();
+
+    default void init() {
+        reset();
+    }

Review Comment:
   What's the usage of this method?



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/AggregateFunction.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.store.file.mergetree.compact;
+
+import java.io.Serializable;
+
+/**
+ * 自定义的列聚合抽象类.
+ *
+ * @param <T>
+ */
+public interface AggregateFunction<T> extends Serializable {
+    //     T aggregator;
+
+    T getResult();
+
+    default void init() {
+        reset();
+    }
+
+    void reset();
+
+    default void aggregate(Object value) {
+        aggregate(value, true);
+    }
+
+    void aggregate(Object value, boolean add);

Review Comment:
   Split this into two methods, `aggregate(Object value)` and `retract(Object 
value)`.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/AggregationMergeFunction.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.store.file.mergetree.compact;
+
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A {@link MergeFunction} where key is primary key (unique) and value is the 
partial record, update
+ * non-null fields on merge.
+ */
+@SuppressWarnings("checkstyle:RegexpSingleline")
+public class AggregationMergeFunction implements MergeFunction {
+
+    private static final long serialVersionUID = 1L;
+
+    private final RowData.FieldGetter[] getters;
+
+    private final RowType rowType;
+    private final ArrayList<AggregateFunction<?>> aggregateFunctions;
+    private final boolean[] isPrimaryKey;
+    private final RowType primaryKeyType;
+    private transient GenericRowData row;
+
+    private final Set<String> aggregateColumnNames;
+
+    public AggregationMergeFunction(
+            RowType primaryKeyType, RowType rowType, Set<String> 
aggregateColumnNames) {
+        this.primaryKeyType = primaryKeyType;
+        this.rowType = rowType;
+        this.aggregateColumnNames = aggregateColumnNames;
+
+        List<LogicalType> fieldTypes = rowType.getChildren();
+        this.getters = new RowData.FieldGetter[fieldTypes.size()];
+        for (int i = 0; i < fieldTypes.size(); i++) {
+            getters[i] = RowData.createFieldGetter(fieldTypes.get(i), i);
+        }
+
+        this.isPrimaryKey = new boolean[this.getters.length];
+        Arrays.fill(isPrimaryKey, false);
+        List<String> rowNames = rowType.getFieldNames();
+        for (String primaryKeyName : primaryKeyType.getFieldNames()) {
+            isPrimaryKey[rowNames.indexOf(primaryKeyName)] = true;
+        }
+
+        this.aggregateFunctions = new ArrayList<>(rowType.getFieldCount());
+        for (int i = 0; i < rowType.getFieldCount(); i++) {
+            AggregateFunction<?> f = null;
+            if (aggregateColumnNames.contains(rowNames.get(i))) {
+                f = 
choiceRightAggregateFunction(rowType.getTypeAt(i).getDefaultConversion());
+            }
+            aggregateFunctions.add(f);
+        }
+    }
+
+    private AggregateFunction<?> choiceRightAggregateFunction(Class<?> c) {
+        AggregateFunction<?> f = null;
+        if (Double.class.equals(c)) {
+            f = new DoubleAggregateFunction();
+        } else if (Long.class.equals(c)) {
+            f = new LongAggregateFunction();
+        } else if (Integer.class.equals(c)) {
+            f = new IntegerAggregateFunction();
+        } else if (Float.class.equals(c)) {
+            f = new FloatAggregateFunction();
+        }
+        return f;
+    }

Review Comment:
   A better implementation is to have a factory or util class which reads in a 
aggregate function name and its type and gives out the corresponding function.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/AggregationMergeFunction.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.store.file.mergetree.compact;
+
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A {@link MergeFunction} where key is primary key (unique) and value is the 
partial record, update
+ * non-null fields on merge.
+ */
+@SuppressWarnings("checkstyle:RegexpSingleline")
+public class AggregationMergeFunction implements MergeFunction {
+
+    private static final long serialVersionUID = 1L;
+
+    private final RowData.FieldGetter[] getters;
+
+    private final RowType rowType;
+    private final ArrayList<AggregateFunction<?>> aggregateFunctions;
+    private final boolean[] isPrimaryKey;
+    private final RowType primaryKeyType;
+    private transient GenericRowData row;
+
+    private final Set<String> aggregateColumnNames;
+
+    public AggregationMergeFunction(
+            RowType primaryKeyType, RowType rowType, Set<String> 
aggregateColumnNames) {
+        this.primaryKeyType = primaryKeyType;
+        this.rowType = rowType;
+        this.aggregateColumnNames = aggregateColumnNames;
+
+        List<LogicalType> fieldTypes = rowType.getChildren();
+        this.getters = new RowData.FieldGetter[fieldTypes.size()];
+        for (int i = 0; i < fieldTypes.size(); i++) {
+            getters[i] = RowData.createFieldGetter(fieldTypes.get(i), i);
+        }
+
+        this.isPrimaryKey = new boolean[this.getters.length];
+        Arrays.fill(isPrimaryKey, false);

Review Comment:
   No need. `false` is the default value.



-- 
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: issues-unsubscr...@flink.apache.org

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

Reply via email to