lowka commented on code in PR #2463:
URL: https://github.com/apache/ignite-3/pull/2463#discussion_r1299835905


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/DestinationFactory.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.ignite.internal.sql.engine.exec;
+
+import static org.apache.ignite.internal.util.CollectionUtils.first;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.sql.engine.metadata.ColocationGroup;
+import org.apache.ignite.internal.sql.engine.metadata.NodeWithTerm;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.AllNodes;
+import org.apache.ignite.internal.sql.engine.trait.Destination;
+import org.apache.ignite.internal.sql.engine.trait.DistributionFunction;
+import 
org.apache.ignite.internal.sql.engine.trait.DistributionFunction.AffinityDistribution;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistribution;
+import org.apache.ignite.internal.sql.engine.trait.Partitioned;
+import org.apache.ignite.internal.sql.engine.trait.RandomNode;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.sql.engine.util.HashFunctionFactory;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Factory that resolves {@link IgniteDistribution} trait, which represents 
logical {@link DistributionFunction} function, into its
+ * physical representation - {@link Destination} function.
+ */
+public class DestinationFactory<RowT> {
+    private final HashFunctionFactory<RowT> hashFunctionFactory;
+    private final ResolvedDependencies dependencies;
+
+    /**
+     * Constructor.
+     *
+     * @param hashFunctionFactory Hash-function factory required to resolve 
hash-based distributions.
+     * @param dependencies Dependencies required to resolve row value 
dependent distributions.
+     */
+    DestinationFactory(HashFunctionFactory<RowT> hashFunctionFactory, 
ResolvedDependencies dependencies) {
+        this.hashFunctionFactory = hashFunctionFactory;
+        this.dependencies = dependencies;
+    }
+
+    /**
+     * Creates a destination based on given distribution and nodes mapping.
+     *
+     * @param distribution Distribution function.
+     * @param group Target mapping.
+     * @return Destination function.
+     */
+    Destination<RowT> createDestination(IgniteDistribution distribution, 
ColocationGroup group) {
+        DistributionFunction function = distribution.function();
+        ImmutableIntList keys = distribution.getKeys();
+
+        switch (function.type()) {
+            case SINGLETON:
+                if (group == null || group.nodeNames() == null || 
group.nodeNames().size() != 1) {

Review Comment:
   I think it would be better to move argument checks of each destination type 
to appropriate constructor so that it would not be possible create an invalid 
destination. There is no penalty in doing so, since there is only one place 
where these destination are used they do not cross serialisation boundaries.



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/DestinationFactory.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.ignite.internal.sql.engine.exec;
+
+import static org.apache.ignite.internal.util.CollectionUtils.first;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.sql.engine.metadata.ColocationGroup;
+import org.apache.ignite.internal.sql.engine.metadata.NodeWithTerm;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.AllNodes;
+import org.apache.ignite.internal.sql.engine.trait.Destination;
+import org.apache.ignite.internal.sql.engine.trait.DistributionFunction;
+import 
org.apache.ignite.internal.sql.engine.trait.DistributionFunction.AffinityDistribution;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistribution;
+import org.apache.ignite.internal.sql.engine.trait.Partitioned;
+import org.apache.ignite.internal.sql.engine.trait.RandomNode;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.sql.engine.util.HashFunctionFactory;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Factory that resolves {@link IgniteDistribution} trait, which represents 
logical {@link DistributionFunction} function, into its
+ * physical representation - {@link Destination} function.
+ */
+public class DestinationFactory<RowT> {

Review Comment:
   I think this class can be package private because it is only used by 
`LogicalRelImplementor`.



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/DestinationFactory.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.ignite.internal.sql.engine.exec;
+
+import static org.apache.ignite.internal.util.CollectionUtils.first;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.sql.engine.metadata.ColocationGroup;
+import org.apache.ignite.internal.sql.engine.metadata.NodeWithTerm;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.AllNodes;
+import org.apache.ignite.internal.sql.engine.trait.Destination;
+import org.apache.ignite.internal.sql.engine.trait.DistributionFunction;
+import 
org.apache.ignite.internal.sql.engine.trait.DistributionFunction.AffinityDistribution;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistribution;
+import org.apache.ignite.internal.sql.engine.trait.Partitioned;
+import org.apache.ignite.internal.sql.engine.trait.RandomNode;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.sql.engine.util.HashFunctionFactory;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Factory that resolves {@link IgniteDistribution} trait, which represents 
logical {@link DistributionFunction} function, into its
+ * physical representation - {@link Destination} function.
+ */
+public class DestinationFactory<RowT> {
+    private final HashFunctionFactory<RowT> hashFunctionFactory;
+    private final ResolvedDependencies dependencies;
+
+    /**
+     * Constructor.
+     *
+     * @param hashFunctionFactory Hash-function factory required to resolve 
hash-based distributions.
+     * @param dependencies Dependencies required to resolve row value 
dependent distributions.
+     */
+    DestinationFactory(HashFunctionFactory<RowT> hashFunctionFactory, 
ResolvedDependencies dependencies) {
+        this.hashFunctionFactory = hashFunctionFactory;
+        this.dependencies = dependencies;
+    }
+
+    /**
+     * Creates a destination based on given distribution and nodes mapping.
+     *
+     * @param distribution Distribution function.
+     * @param group Target mapping.
+     * @return Destination function.
+     */
+    Destination<RowT> createDestination(IgniteDistribution distribution, 
ColocationGroup group) {
+        DistributionFunction function = distribution.function();
+        ImmutableIntList keys = distribution.getKeys();
+
+        switch (function.type()) {
+            case SINGLETON:
+                if (group == null || group.nodeNames() == null || 
group.nodeNames().size() != 1) {
+                    throw new IllegalStateException();
+                }
+
+                return new 
AllNodes<>(Collections.singletonList(Objects.requireNonNull(first(group.nodeNames()))));
+            case BROADCAST_DISTRIBUTED:
+                if (group == null || nullOrEmpty(group.nodeNames())) {
+                    throw new IllegalStateException();
+                }
+
+                return new AllNodes<>(group.nodeNames());
+            case RANDOM_DISTRIBUTED:
+                if (group == null || nullOrEmpty(group.nodeNames())) {
+                    throw new IllegalStateException();
+                }
+
+                return new RandomNode<>(group.nodeNames());
+            case HASH_DISTRIBUTED: {
+                if (group == null || nullOrEmpty(group.assignments()) || 
keys.isEmpty()) {
+                    throw new IllegalStateException();
+                }
+
+                List<List<String>> assignments = 
Commons.transform(group.assignments(), v -> Commons.transform(v, 
NodeWithTerm::name));
+
+                if (IgniteUtils.assertionsEnabled()) {
+                    for (List<String> assignment : assignments) {
+                        assert nullOrEmpty(assignment) || assignment.size() == 
1;
+                    }
+                }
+
+                if (function.affinity()) {
+                    int tableId = ((AffinityDistribution) function).tableId();
+
+                    TableDescriptor tableDescriptor = 
dependencies.updatableTable(tableId).descriptor();

Review Comment:
   I think there is a ticket to remove updatable table from 
`ResolvedDependencies`, if there is, it would be better to hide 
`TableDescriptor` behind a method that returns `TableDescriptor ` by `tableId`. 
What do you think?



-- 
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]

Reply via email to