From 1fd21f05fa7f288e42d19126f4d4ea9a3a7f902f Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Tue, 16 Jan 2018 18:55:04 +0300 Subject: [PATCH 01/38] IGNITE-7437 Initial version of d-learn framework in ML module. --- .../ignite/ml/dlearn/DLearnContext.java | 91 +++++++++++++ .../ml/dlearn/DLearnContextFactory.java | 35 +++++ .../ml/dlearn/DLearnPartitionFactory.java | 38 ++++++ .../ml/dlearn/DLearnPartitionStorage.java | 49 +++++++ .../context/cache/CacheDLearnContext.java | 125 ++++++++++++++++++ .../cache/CacheDLearnContextFactory.java | 108 +++++++++++++++ .../context/cache/CacheDLearnPartition.java | 57 ++++++++ .../cache/CacheDLearnPartitionStorage.java | 84 ++++++++++++ .../ml/dlearn/context/cache/package-info.java | 22 +++ .../DLearnPartitionAffinityFunction.java | 96 ++++++++++++++ .../context/local/LocalDLearnContext.java | 91 +++++++++++++ .../local/LocalDLearnContextFactory.java | 70 ++++++++++ .../context/local/LocalDLearnPartition.java | 47 +++++++ .../local/LocalDLearnPartitionStorage.java | 52 ++++++++ .../ml/dlearn/context/local/package-info.java | 22 +++ .../ml/dlearn/context/package-info.java | 22 +++ .../ml/dlearn/dataset/DatasetMathUtils.java | 91 +++++++++++++ .../apache/ignite/ml/dlearn/package-info.java | 22 +++ .../ml/dlearn/part/DatasetDLeanPartition.java | 75 +++++++++++ .../part/LabeledDatasetDLearnPartition.java | 56 ++++++++ ...acheDatasetDLearnPartitionTransformer.java | 89 +++++++++++++ ...eledDatasetDLearnPartitionTransformer.java | 99 ++++++++++++++ ...ocalDatasetDLearnPartitionTransformer.java | 64 +++++++++ ...eledDatasetDLearnPartitionTransformer.java | 52 ++++++++ .../ignite/ml/dlearn/part/package-info.java | 22 +++ .../utils/DLearnContextPartitionKey.java | 93 +++++++++++++ .../ignite/ml/dlearn/utils/package-info.java | 22 +++ .../linear/LinearRegressionLSQRTrainer.java | 84 ++++++++++++ .../java/org/apache/ignite/ml/Playground.java | 119 +++++++++++++++++ 29 files changed, 1897 insertions(+) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContextFactory.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionFactory.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionStorage.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartitionStorage.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunction.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartitionStorage.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DatasetMathUtils.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/DatasetDLeanPartition.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/LabeledDatasetDLearnPartition.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/cache/CacheDatasetDLearnPartitionTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/cache/CacheLabeledDatasetDLearnPartitionTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/local/LocalDatasetDLearnPartitionTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/local/LocalLabeledDatasetDLearnPartitionTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextPartitionKey.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/Playground.java diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java new file mode 100644 index 0000000000000..be80e4c8d9a29 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java @@ -0,0 +1,91 @@ +/* + * 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.ml.dlearn; + +import org.apache.ignite.ml.math.functions.IgniteBiConsumer; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; +import org.apache.ignite.ml.math.functions.IgniteConsumer; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +/** + * Learning context is a context maintained during a whole learning process. The context provides an ability to perform + * calculations in map-reduce manner and guarantees maintenance of the partition states between compute calls. + * + * @param

type of learning context partition + */ +public interface DLearnContext

{ + /** + * Computes a given function on every d-learn partition in current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param mapper mapper function applied on every partition + * @param reducer reducer of the results + * @param result type + * @return final reduced result + */ + public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer); + + /** + * Computes a given function on every d-learn partition in current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param mapper mapper function applied on every partition + * @param reducer reducer of the results + * @param result type + * @return final reduced result + */ + default public R compute(IgniteFunction mapper, IgniteBinaryOperator reducer) { + return compute((part, partIdx) -> mapper.apply(part), reducer); + } + + /** + * Computes a given function on every d-learn partition in current learning context independently. The goal of this + * approach is to perform {@code mapper} locally on the nodes where partitions are placed and do not involve network + * subsystem where it's possible. + * + * @param mapper mapper function applied on every partition + */ + public void compute(IgniteBiConsumer mapper); + + /** + * Computes a given function on every d-learn partition in current learning context independently. The goal of this + * approach is to perform {@code mapper} locally on the nodes where partitions are placed and do not involve network + * subsystem where it's possible. + * + * @param mapper mapper function applied on every partition + */ + default public void compute(IgniteConsumer

mapper) { + compute((part, partIdx) -> mapper.accept(part)); + } + + /** + * Transforms current learning context into another learning context which contains another type of d-learn + * partitions. Transformation doesn't involve new cache instantiation or network data transfer, it just performs + * {@code #transform(IgniteBiConsumer, DLearnPartitionFactory)} locally on every partition in the current context + * and saves results into the same context cache, but with a new context id. + * + * @param transformer transformer function which makes the new d-learn partition from the old one + * @param partFactory d-learn partition factory + * @param type of the new d-learn partition + * @return new learning context + */ + public DLearnContext transform(IgniteBiConsumer transformer, DLearnPartitionFactory partFactory); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContextFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContextFactory.java new file mode 100644 index 0000000000000..6ceec65b4252e --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContextFactory.java @@ -0,0 +1,35 @@ +/* + * 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.ml.dlearn; + +import java.io.Serializable; + +/** + * Factory produces learning context instances. + * + * @param

type of d-learn partition + */ +@FunctionalInterface +public interface DLearnContextFactory

extends Serializable { + /** + * Creates new instance of learning context. + * + * @return learning context + */ + public DLearnContext

createContext(); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionFactory.java new file mode 100644 index 0000000000000..f99226fb5d374 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionFactory.java @@ -0,0 +1,38 @@ +/* + * 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.ml.dlearn; + +import java.io.Serializable; + +/** + * With assumption that a d-lear partition is the type-safe wrapper on top of a partition storage (distributed or not), + * {@code DLearnPartitionFactory} provides API for instantiation of a new d-learn partition instances based on given + * partition storage. + * + * @param

type of d-learn partition + */ +@FunctionalInterface +public interface DLearnPartitionFactory

extends Serializable { + /** + * Creates a new d-learn partition (type-safe wrapper on top of a partition storage). + * + * @param storage partition storage + * @return d-learn partition + */ + public P createPartition(DLearnPartitionStorage storage); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionStorage.java new file mode 100644 index 0000000000000..b375f5140f6f3 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionStorage.java @@ -0,0 +1,49 @@ +/* + * 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.ml.dlearn; + +/** + * D-learn partition storage is a common interface for partition storages both local and distributed. it allows to save, + * retrieve and remove objects identified by keys from the d-learn partition. + */ +public interface DLearnPartitionStorage { + /** + * Saves given value in the d-learn partition with given key. + * + * @param key key + * @param val value + * @param type of value + */ + public void put(String key, T val); + + /** + * Retrieves value from the d-learn partition by given key. + * + * @param key key + * @param type of value + * @return value + */ + public T get(String key); + + /** + * Removes value from the d-learn partition by given key. + * + * @param key key + */ + public void remove(String key); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java new file mode 100644 index 0000000000000..aa1b1d5742524 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java @@ -0,0 +1,125 @@ +/* + * 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.ml.dlearn.context.cache; + +import java.util.Collection; +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cluster.ClusterGroup; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.ml.dlearn.DLearnContext; +import org.apache.ignite.ml.dlearn.DLearnPartitionFactory; +import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; +import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; +import org.apache.ignite.ml.math.functions.IgniteBiConsumer; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; + +/** + * Learning context based in Ignite Cache storage. + * + * @param

type of learning context partition + */ +public class CacheDLearnContext

implements DLearnContext

{ + /** */ + private final Ignite ignite; + + /** */ + private final String learningCtxCacheName; + + /** */ + private final DLearnPartitionFactory

partFactory; + + /** */ + private final UUID learningCtxId; + + /** */ + public CacheDLearnContext(Ignite ignite, String learningCtxCacheName, DLearnPartitionFactory

partFactory, UUID learningCtxId) { + this.ignite = ignite; + this.learningCtxCacheName = learningCtxCacheName; + this.partFactory = partFactory; + this.learningCtxId = learningCtxId; + } + + /** */ + public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer) { + ClusterGroup clusterGrp = ignite.cluster().forDataNodes(learningCtxCacheName); + + Collection results = ignite.compute(clusterGrp).broadcast(() -> { + IgniteCache learningCtxCache = ignite.cache(learningCtxCacheName); + + Affinity affinity = ignite.affinity(learningCtxCacheName); + ClusterNode locNode = ignite.cluster().localNode(); + + int[] partitions = affinity.primaryPartitions(locNode); + R res = null; + for (int part : partitions) { + DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, learningCtxId, part); + P learningCtxPart = partFactory.createPartition(storage); + R partRes = mapper.apply(learningCtxPart, part); + res = reducer.apply(res, partRes); + } + return res; + }); + + return reduce(results, reducer); + } + + /** */ + @Override public void compute(IgniteBiConsumer mapper) { + ClusterGroup clusterGrp = ignite.cluster().forDataNodes(learningCtxCacheName); + + ignite.compute(clusterGrp).broadcast(() -> { + IgniteCache learningCtxCache = ignite.cache(learningCtxCacheName); + + Affinity affinity = ignite.affinity(learningCtxCacheName); + ClusterNode locNode = ignite.cluster().localNode(); + + int[] partitions = affinity.primaryPartitions(locNode); + for (int part : partitions) { + DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, learningCtxId, part); + P learningCtxPart = partFactory.createPartition(storage); + mapper.accept(learningCtxPart, part); + } + }); + } + + /** */ + @Override public DLearnContext transform(IgniteBiConsumer transformer, DLearnPartitionFactory partFactory) { + UUID newLearningCtxId = UUID.randomUUID(); + + compute((part, partIdx) -> { + IgniteCache learningCtxCache = ignite.cache(learningCtxCacheName); + DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, newLearningCtxId, partIdx); + T newPart = partFactory.createPartition(storage); + transformer.accept(part, newPart); + }); + + return new CacheDLearnContext<>(ignite, learningCtxCacheName, partFactory, newLearningCtxId); + } + + /** */ + private R reduce(Collection results, IgniteBinaryOperator reducer) { + R res = null; + for (R partRes : results) + res = reducer.apply(res, partRes); + return res; + } +} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java new file mode 100644 index 0000000000000..55c96f0f4ac55 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java @@ -0,0 +1,108 @@ +/* + * 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.ml.dlearn.context.cache; + +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.ml.dlearn.DLearnContextFactory; +import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; +import org.apache.ignite.ml.dlearn.context.cache.utils.DLearnPartitionAffinityFunction; +import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; + +/** + * Factory produces cache learning context by extracting data from other cache. + * + * @param type of keys in upstream cache + * @param type of values in upstream values + */ +public class CacheDLearnContextFactory implements DLearnContextFactory> { + /** */ + private static final long serialVersionUID = 2903867793242785702L; + + /** */ + private static final String LEARNING_CONTEXT_CACHE_NAME = "%s_LEARNING_CONTEXT_%s"; + + /** */ + private final Ignite ignite; + + /** */ + private final IgniteCache upstreamCache; + + /** */ + public CacheDLearnContextFactory(Ignite ignite, IgniteCache upstreamCache) { + this.ignite = ignite; + this.upstreamCache = upstreamCache; + } + + /** {@inheritDoc} */ + @Override public CacheDLearnContext> createContext() { + CacheConfiguration learningCtxCacheCfg = new CacheConfiguration<>(); + learningCtxCacheCfg.setName(String.format(LEARNING_CONTEXT_CACHE_NAME, upstreamCache, UUID.randomUUID())); + learningCtxCacheCfg.setAffinity(createLearningContextCacheAffinityFunction()); + + IgniteCache learningCtxCache = ignite.createCache(learningCtxCacheCfg); + + Affinity affinity = ignite.affinity(upstreamCache.getName()); + UUID learningCtxId = UUID.randomUUID(); + for (int partIdx = 0; partIdx < affinity.partitions(); partIdx++) { + DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, learningCtxId, partIdx); + CacheDLearnPartition part = new CacheDLearnPartition<>(storage); + part.setUpstreamCacheName(upstreamCache.getName()); + part.setPart(partIdx); + } + + return new CacheDLearnContext<>(ignite, learningCtxCache.getName(), CacheDLearnPartition::new, learningCtxId); + } + + /** + * Creates learning context cache affinity function based in upstream cache. This function retrieves current + * topology version and layout (partition-to-node map) of upstream cache corresponding to it, then applies this + * retrieved layout to create an affinity function for learning context cache. + * + * @return affinity function + */ + private DLearnPartitionAffinityFunction createLearningContextCacheAffinityFunction() { + Affinity affinity = ignite.affinity(upstreamCache.getName()); + + // tries to collect partition-to-node map and checks that topology version hasn't been changed during this + // process + List initAssignment; + long topVer; + while (true) { + topVer = ignite.cluster().topologyVersion(); + + initAssignment = new ArrayList<>(affinity.partitions()); + for (int part = 0; part < affinity.partitions(); part++) { + ClusterNode primaryNode = affinity.mapPartitionToNode(part); + initAssignment.add(primaryNode.id()); + } + + // if topology version changed we need to try again + if (topVer == ignite.cluster().topologyVersion()) + break; + } + + return new DLearnPartitionAffinityFunction(initAssignment, topVer); + } +} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java new file mode 100644 index 0000000000000..b74e24ed88c82 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java @@ -0,0 +1,57 @@ +/* + * 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.ml.dlearn.context.cache; + +import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; + +/** */ +public class CacheDLearnPartition { + /** */ + private static final String UPSTREAM_CACHE_NAME_KEY = "upstream_cache_name"; + + /** */ + private static final String PART_KEY = "part"; + + /** */ + private final DLearnPartitionStorage storage; + + /** */ + public CacheDLearnPartition(DLearnPartitionStorage storage) { + this.storage = storage; + } + + /** */ + public void setUpstreamCacheName(String upstreamCacheName) { + storage.put(UPSTREAM_CACHE_NAME_KEY, upstreamCacheName); + } + + /** */ + public String getUpstreamCacheName() { + return storage.get(UPSTREAM_CACHE_NAME_KEY); + } + + /** */ + public void setPart(int part) { + storage.put(PART_KEY, part); + } + + /** */ + public int getPart() { + return storage.get(PART_KEY); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartitionStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartitionStorage.java new file mode 100644 index 0000000000000..c9b771ab1e8a1 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartitionStorage.java @@ -0,0 +1,84 @@ +/* + * 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.ml.dlearn.context.cache; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.UUID; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; +import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; + +public class CacheDLearnPartitionStorage implements DLearnPartitionStorage { + + private final IgniteCache learningCtxCache; + + private final UUID learningCtxId; + + private final int part; + + public CacheDLearnPartitionStorage(IgniteCache learningCtxCache, UUID learningCtxId, int part) { + this.learningCtxCache = learningCtxCache; + this.learningCtxId = learningCtxId; + this.part = part; + } + + @Override public void put(String key, T val) { + learningCtxCache.put(new DLearnContextPartitionKey(part, learningCtxId, key), serialize(val)); + } + + @SuppressWarnings("unchecked") + @Override public T get(String key) { + return (T) deserialize(learningCtxCache.localPeek(new DLearnContextPartitionKey(part, learningCtxId, key))); + } + + @Override public void remove(String key) { + learningCtxCache.remove(new DLearnContextPartitionKey(part, learningCtxId, key)); + } + + /** */ + private byte[] serialize(Object obj) { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos)) { + oos.writeObject(obj); + + return baos.toByteArray(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** */ + private Object deserialize(byte[] arr) { + if (arr == null) + return null; + + try (ByteArrayInputStream bais = new ByteArrayInputStream(arr); + ObjectInputStream ois = new ObjectInputStream(bais)) { + return ois.readObject(); + } + catch (IOException | ClassNotFoundException e) { + throw new RuntimeException(e); + } + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/package-info.java new file mode 100644 index 0000000000000..149e694fae31a --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Root package for distributed learning implementations based on Ignite Cache. + */ +package org.apache.ignite.ml.dlearn.context.cache; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunction.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunction.java new file mode 100644 index 0000000000000..dac086f3ed475 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunction.java @@ -0,0 +1,96 @@ +/* + * 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.ml.dlearn.context.cache.utils; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.cache.affinity.AffinityFunction; +import org.apache.ignite.cache.affinity.AffinityFunctionContext; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; + +/** + * Affinity function used to identify partition number and node to place learning context partition. This function is + * initialized with {@link #initAssignment} parameter which contains information about upstream cache distribution across + * cluster. It allows it to place learning context partitions on the same nodes as partitions of the upstream cache. + */ +public class DLearnPartitionAffinityFunction implements AffinityFunction { + /** */ + private static final long serialVersionUID = 7735390384525189270L; + + /** + * Initial distribution of the partitions (copy of upstream cache partitions distribution) + */ + private final List initAssignment; + + /** + * Version of the topology used to make an {@link #initAssignment} + */ + private final long initTopVer; + + /** */ + public DLearnPartitionAffinityFunction(List initAssignment, long initTopVer) { + this.initAssignment = initAssignment; + this.initTopVer = initTopVer; + } + + /** {@inheritDoc} */ + @Override public void reset() { + // do nothing + } + + /** {@inheritDoc} */ + @Override public int partitions() { + return initAssignment.size(); + } + + /** {@inheritDoc} */ + @Override public int partition(Object key) { + BinaryObject bo = (BinaryObject)key; + DLearnContextPartitionKey datasetPartKey = bo.deserialize(); + return datasetPartKey.getPart(); + } + + /** {@inheritDoc} */ + @Override public List> assignPartitions(AffinityFunctionContext affCtx) { + long currTopVer = affCtx.currentTopologyVersion().topologyVersion(); + List> assignment = new ArrayList<>(initAssignment.size()); + if (currTopVer == initTopVer) { + Map topSnapshotIdx = new HashMap<>(); + for (ClusterNode node : affCtx.currentTopologySnapshot()) + topSnapshotIdx.put(node.id(), node); + for (int part = 0; part < initAssignment.size(); part++) { + UUID partNodeId = initAssignment.get(part); + ClusterNode partNode = topSnapshotIdx.get(partNodeId); + assignment.add(Collections.singletonList(partNode)); + } + return assignment; + } + throw new IllegalStateException(); + } + + /** {@inheritDoc} */ + @Override public void removeNode(UUID nodeId) { + // do nothing + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java new file mode 100644 index 0000000000000..b36510aed61bd --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java @@ -0,0 +1,91 @@ +/* + * 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.ml.dlearn.context.local; + +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.ml.dlearn.DLearnContext; +import org.apache.ignite.ml.dlearn.DLearnPartitionFactory; +import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; +import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; +import org.apache.ignite.ml.math.functions.IgniteBiConsumer; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; + +/** + * Learning context based on a local on-heap storage. + * + * @param

type of learning context partition + */ +public class LocalDLearnContext

implements DLearnContext

{ + /** */ + private final Map learningCtxMap; + + /** */ + private final DLearnPartitionFactory

partFactory; + + /** */ + private final UUID learningCtxId; + + /** */ + private final int partitions; + + /** */ + public LocalDLearnContext(Map learningCtxMap, + DLearnPartitionFactory

partFactory, UUID learningCtxId, int partitions) { + this.learningCtxMap = learningCtxMap; + this.partFactory = partFactory; + this.learningCtxId = learningCtxId; + this.partitions = partitions; + } + + /** */ + @Override public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer) { + R res = null; + for (int partIdx = 0; partIdx < partitions; partIdx++) { + DLearnPartitionStorage storage = new LocalDLearnPartitionStorage(learningCtxMap, learningCtxId, partIdx); + P part = partFactory.createPartition(storage); + R partRes = mapper.apply(part, partIdx); + res = reducer.apply(res, partRes); + } + return res; + } + + /** */ + @Override public void compute(IgniteBiConsumer mapper) { + for (int partIdx = 0; partIdx < partitions; partIdx++) { + DLearnPartitionStorage storage = new LocalDLearnPartitionStorage(learningCtxMap, learningCtxId, partIdx); + P part = partFactory.createPartition(storage); + mapper.accept(part, partIdx); + } + } + + /** */ + @Override public DLearnContext transform(IgniteBiConsumer transformer, + DLearnPartitionFactory partFactory) { + UUID newLearningCtxId = UUID.randomUUID(); + + compute((part, partIdx) -> { + DLearnPartitionStorage newStorage = new LocalDLearnPartitionStorage(learningCtxMap, newLearningCtxId, partIdx); + T newPart = partFactory.createPartition(newStorage); + transformer.accept(part, newPart); + }); + + return new LocalDLearnContext<>(learningCtxMap, partFactory, newLearningCtxId, partitions); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java new file mode 100644 index 0000000000000..5945b9c788c5f --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java @@ -0,0 +1,70 @@ +/* + * 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.ml.dlearn.context.local; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.ml.dlearn.DLearnContextFactory; +import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; +import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; + +/** + * Factory produces local learning context by extracting data from local list of values. + * + * @param type of upstream values + */ +public class LocalDLearnContextFactory implements DLearnContextFactory> { + /** */ + private static final long serialVersionUID = -7614441997952907675L; + + /** */ + private final List data; + + /** */ + private final int partitions; + + /** */ + public LocalDLearnContextFactory(List data, int partitions) { + this.data = data; + this.partitions = partitions; + } + + /** {@inheritDoc} */ + @Override public LocalDLearnContext> createContext() { + Map learningCtxMap = new HashMap<>(); + UUID learningCtxId = UUID.randomUUID(); + + int partSize = data.size() / partitions; + + // loads data into learning context partitions + for (int partIdx = 0; partIdx < partitions; partIdx++) { + List partData = new ArrayList<>(); + for (int j = partIdx * partSize; j < (partIdx + 1) * partSize && j < data.size(); j++) + partData.add(data.get(j)); + + DLearnPartitionStorage storage = new LocalDLearnPartitionStorage(learningCtxMap, learningCtxId, partIdx); + LocalDLearnPartition part = new LocalDLearnPartition<>(storage); + part.setPartData(partData); + } + + return new LocalDLearnContext<>(learningCtxMap, LocalDLearnPartition::new, learningCtxId, partitions); + } +} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java new file mode 100644 index 0000000000000..664929299d76d --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java @@ -0,0 +1,47 @@ +/* + * 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.ml.dlearn.context.local; + +import java.util.List; +import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; + +/** + * Learning context partition which uses local on-heap hash map to keep data. + */ +public class LocalDLearnPartition { + /** */ + private static final String PART_DATA_KEY = "part_data"; + + /** */ + private final DLearnPartitionStorage storage; + + /** */ + public LocalDLearnPartition(DLearnPartitionStorage storage) { + this.storage = storage; + } + + /** */ + public List getPartData() { + return storage.get(PART_DATA_KEY); + } + + /** */ + public void setPartData(List partData) { + storage.put(PART_DATA_KEY, partData); + } +} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartitionStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartitionStorage.java new file mode 100644 index 0000000000000..07759cb3bd6ab --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartitionStorage.java @@ -0,0 +1,52 @@ +/* + * 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.ml.dlearn.context.local; + +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; +import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; + +public class LocalDLearnPartitionStorage implements DLearnPartitionStorage { + + private final Map learningCtxMap; + + private final UUID learningCtxId; + + private final int part; + + public LocalDLearnPartitionStorage( + Map learningCtxMap, UUID learningCtxId, int part) { + this.learningCtxMap = learningCtxMap; + this.learningCtxId = learningCtxId; + this.part = part; + } + + @Override public void put(String key, T val) { + learningCtxMap.put(new DLearnContextPartitionKey(part, learningCtxId, key), val); + } + + @SuppressWarnings("unchecked") + @Override public T get(String key) { + return (T) learningCtxMap.get(new DLearnContextPartitionKey(part, learningCtxId, key)); + } + + @Override public void remove(String key) { + learningCtxMap.remove(new DLearnContextPartitionKey(part, learningCtxId, key)); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/package-info.java new file mode 100644 index 0000000000000..35a3554acd741 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Root package for distributed learning implementations based on local storage. + */ +package org.apache.ignite.ml.dlearn.context.local; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/package-info.java new file mode 100644 index 0000000000000..dc961d2b1ebc0 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Root package for distributed learning implementations (based on cache, local storage, etc...). + */ +package org.apache.ignite.ml.dlearn.context; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DatasetMathUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DatasetMathUtils.java new file mode 100644 index 0000000000000..7406ae2c92510 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DatasetMathUtils.java @@ -0,0 +1,91 @@ +/* + * 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.ml.dlearn.dataset; + +import com.github.fommil.netlib.BLAS; +import org.apache.ignite.ml.dlearn.DLearnContext; +import org.apache.ignite.ml.dlearn.part.DatasetDLeanPartition; + +/** */ +public class DatasetMathUtils { + /** */ + private static final BLAS blas = BLAS.getInstance(); + + /** */ + public static double[] mean(DLearnContext learningCtx, int[] cols) { + ValueWithCount res = learningCtx.compute((part, partIdx) -> { + double[] features = part.getFeatures(); + int m = part.getRows(); + double[] y = new double[cols.length]; + for (int i = 0; i < cols.length; i++) + for (int j = cols[i] * m; j < (cols[i] + 1) * m; j++) + y[i] += features[j]; + return new ValueWithCount<>(y, m); + }, (a, b) -> a == null ? b : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); + blas.dscal(res.val.length, 1.0 / res.cnt, res.val, 1); + return res.val; + } + + /** */ + public static double mean(DLearnContext learningCtx, int col) { + return mean(learningCtx, new int[]{col})[0]; + } + + /** */ + public static double[] std(DLearnContext learningCtx, int[] cols) { + double[] mean = mean(learningCtx, cols); + ValueWithCount res = learningCtx.compute(part -> { + double[] features = part.getFeatures(); + int m = part.getRows(); + double[] y = new double[cols.length]; + for (int i = 0; i < cols.length; i++) + for (int j = cols[i] * m; j < (cols[i] + 1) * m; j++) + y[i] += Math.pow(features[j] - mean[cols[i]], 2); + return new ValueWithCount<>(y, m); + }, (a, b) -> a == null ? b : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); + blas.dscal(res.val.length, 1.0 / res.cnt, res.val, 1); + for (int i = 0; i < res.val.length; i++) + res.val[i] = Math.sqrt(res.val[i]); + return res.val; + } + + public static double std(DLearnContext learningCtx, int col) { + return std(learningCtx, new int[]{col})[0]; + } + + /** */ + private static double[] sum(double[] a, double[] b) { + blas.daxpy(a.length, 1.0, a, 1, b, 1); + return b; + } + + /** */ + private static class ValueWithCount { + /** */ + private final V val; + + /** */ + private final int cnt; + + /** */ + public ValueWithCount(V val, int cnt) { + this.val = val; + this.cnt = cnt; + } + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/package-info.java new file mode 100644 index 0000000000000..51132cb7b3b36 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Root package for d-learn (distributed learning) framework. + */ +package org.apache.ignite.ml.dlearn; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/DatasetDLeanPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/DatasetDLeanPartition.java new file mode 100644 index 0000000000000..50f00fe476210 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/DatasetDLeanPartition.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.ignite.ml.dlearn.part; + +import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; + +/** + * Interface which provides simple dataset API which allows to get or set an underlying feature matrix in flat format. + */ +public class DatasetDLeanPartition { + /** */ + private static final String FEATURES_KEY = "features"; + + /** */ + private static final String ROWS_KEY = "rows"; + + /** */ + private final DLearnPartitionStorage storage; + + /** */ + public DatasetDLeanPartition(DLearnPartitionStorage storage) { + this.storage = storage; + } + + /** + * Sets matrix of features in flat format. + * + * @param features matrix of features in flat format + */ + public void setFeatures(double[] features) { + storage.put(FEATURES_KEY, features); + } + + /** + * Retrieves matrix of features in flat format. + * + * @return matrix of features in flat format + */ + public double[] getFeatures() { + return storage.get(FEATURES_KEY); + } + + /** + * Sets number of rows. + * + * @param rows number of rows + */ + public void setRows(int rows) { + storage.put(ROWS_KEY, rows); + } + + /** + * Retrieves number of rows. + * + * @return number of rows + */ + public int getRows() { + return storage.get(ROWS_KEY); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/LabeledDatasetDLearnPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/LabeledDatasetDLearnPartition.java new file mode 100644 index 0000000000000..fdaa3e2fbd036 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/LabeledDatasetDLearnPartition.java @@ -0,0 +1,56 @@ +/* + * 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.ml.dlearn.part; + +import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; + +/** + * Interface which provides simple dataset API which allows to get or set an underlying feature matrix in flat format + * and vector of labels. + */ +public class LabeledDatasetDLearnPartition extends DatasetDLeanPartition { + /** */ + private static final String LABELS_KEY = "labels"; + + /** */ + private final DLearnPartitionStorage storage; + + /** */ + public LabeledDatasetDLearnPartition(DLearnPartitionStorage storage) { + super(storage); + this.storage = storage; + } + + /** + * Sets labels. + * + * @param labels labels + */ + public void setLabels(L[] labels) { + storage.put(LABELS_KEY, labels); + } + + /** + * Retrieves labels. + * + * @return labels + */ + public L[] getLabels() { + return storage.get(LABELS_KEY); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/cache/CacheDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/cache/CacheDatasetDLearnPartitionTransformer.java new file mode 100644 index 0000000000000..5e134c9c6fbd0 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/cache/CacheDatasetDLearnPartitionTransformer.java @@ -0,0 +1,89 @@ +/* + * 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.ml.dlearn.part.cache; + +import java.util.ArrayList; +import java.util.List; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; +import org.apache.ignite.ml.dlearn.part.DatasetDLeanPartition; +import org.apache.ignite.ml.math.functions.IgniteBiConsumer; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +/** */ +public class CacheDatasetDLearnPartitionTransformer implements IgniteBiConsumer,DatasetDLeanPartition> { + /** */ + private static final long serialVersionUID = -7398727071330763144L; + + /** */ + private final IgniteBiFunction featureExtractor; + + /** */ + public CacheDatasetDLearnPartitionTransformer(IgniteBiFunction featureExtractor) { + this.featureExtractor = featureExtractor; + } + + /** */ + @Override public void accept(CacheDLearnPartition oldPart, DatasetDLeanPartition newPart) { + List> partData = queryPartDataIntoList(oldPart); + + double[] features = null; + int m = partData.size(), n = 0; + for (int i = 0; i < partData.size(); i++) { + Cache.Entry entry = partData.get(i); + double[] rowFeatures = featureExtractor.apply(entry.getKey(), entry.getValue()); + + if (i == 0) { + n = rowFeatures.length; + features = new double[m * n]; + } + + if (rowFeatures.length != n) + throw new IllegalStateException(); + + for (int j = 0; j < rowFeatures.length; j++) + features[j * m + i] = rowFeatures[j]; + } + newPart.setFeatures(features); + newPart.setRows(m); + } + + /** */ + private List> queryPartDataIntoList(CacheDLearnPartition oldPart) { + List> partData = new ArrayList<>(); + for (Cache.Entry entry : queryPartData(oldPart)) + partData.add(entry); + return partData; + } + + /** */ + private Iterable> queryPartData(CacheDLearnPartition oldPart) { + Ignite ignite = Ignition.localIgnite(); + IgniteCache upstreamCache = ignite.cache(oldPart.getUpstreamCacheName()); + + ScanQuery qry = new ScanQuery<>(); + qry.setLocal(true); + qry.setPartition(oldPart.getPart()); + + return upstreamCache.query(qry); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/cache/CacheLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/cache/CacheLabeledDatasetDLearnPartitionTransformer.java new file mode 100644 index 0000000000000..2a893664f7f4f --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/cache/CacheLabeledDatasetDLearnPartitionTransformer.java @@ -0,0 +1,99 @@ +/* + * 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.ml.dlearn.part.cache; + +import java.util.ArrayList; +import java.util.List; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; +import org.apache.ignite.ml.dlearn.part.LabeledDatasetDLearnPartition; +import org.apache.ignite.ml.math.functions.IgniteBiConsumer; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +/** */ +public class CacheLabeledDatasetDLearnPartitionTransformer implements IgniteBiConsumer, LabeledDatasetDLearnPartition> { + /** */ + private static final long serialVersionUID = 3479218902890029731L; + + /** */ + private final IgniteBiFunction featureExtractor; + + /** */ + private final IgniteBiFunction lbExtractor; + + /** */ + public CacheLabeledDatasetDLearnPartitionTransformer( + IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + this.featureExtractor = featureExtractor; + this.lbExtractor = lbExtractor; + } + + /** */ + @SuppressWarnings("unchecked") + @Override public void accept(CacheDLearnPartition oldPart, LabeledDatasetDLearnPartition newPart) { + List> partData = queryPartDataIntoList(oldPart); + + int m = partData.size(), n = 0; + double[] features = null; + L[] labels = (L[]) new Object[m]; + for (int i = 0; i < partData.size(); i++) { + Cache.Entry entry = partData.get(i); + double[] rowFeatures = featureExtractor.apply(entry.getKey(), entry.getValue()); + labels[i] = lbExtractor.apply(entry.getKey(), entry.getValue()); + + if (i == 0) { + n = rowFeatures.length; + features = new double[m * n]; + } + + if (rowFeatures.length != n) + throw new IllegalStateException(); + + for (int j = 0; j < rowFeatures.length; j++) + features[j * m + i] = rowFeatures[j]; + } + newPart.setFeatures(features); + newPart.setRows(m); + newPart.setLabels(labels); + } + + /** */ + private List> queryPartDataIntoList(CacheDLearnPartition oldPart) { + List> partData = new ArrayList<>(); + for (Cache.Entry entry : queryPartData(oldPart)) + partData.add(entry); + return partData; + } + + /** */ + private Iterable> queryPartData(CacheDLearnPartition oldPart) { + Ignite ignite = Ignition.localIgnite(); + IgniteCache upstreamCache = ignite.cache(oldPart.getUpstreamCacheName()); + + ScanQuery qry = new ScanQuery<>(); + qry.setLocal(true); + qry.setPartition(oldPart.getPart()); + + return upstreamCache.query(qry); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/local/LocalDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/local/LocalDatasetDLearnPartitionTransformer.java new file mode 100644 index 0000000000000..bb178364d1170 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/local/LocalDatasetDLearnPartitionTransformer.java @@ -0,0 +1,64 @@ +/* + * 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.ml.dlearn.part.local; + +import java.util.List; +import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; +import org.apache.ignite.ml.dlearn.part.DatasetDLeanPartition; +import org.apache.ignite.ml.math.functions.IgniteBiConsumer; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +/** */ +public class LocalDatasetDLearnPartitionTransformer + implements IgniteBiConsumer, P> { + /** */ + private static final long serialVersionUID = -7567051002880704559L; + + /** */ + private final IgniteFunction featureExtractor; + + /** */ + public LocalDatasetDLearnPartitionTransformer(IgniteFunction extractor) { + featureExtractor = extractor; + } + + /** */ + @Override public void accept(LocalDLearnPartition oldPart, P newPart) { + List partData = oldPart.getPartData(); + if (partData != null && !partData.isEmpty()) { + double[] features = null; + int m = partData.size(), n = 0; + for (int i = 0; i < partData.size(); i++) { + double[] rowFeatures = featureExtractor.apply(partData.get(i)); + + if (i == 0) { + n = rowFeatures.length; + features = new double[m * n]; + } + + if (rowFeatures.length != n) + throw new IllegalStateException(); + + for (int j = 0; j < rowFeatures.length; j++) + features[j * m + i] = rowFeatures[j]; + } + newPart.setFeatures(features); + newPart.setRows(m); + } + } +} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/local/LocalLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/local/LocalLabeledDatasetDLearnPartitionTransformer.java new file mode 100644 index 0000000000000..89d799f2096ff --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/local/LocalLabeledDatasetDLearnPartitionTransformer.java @@ -0,0 +1,52 @@ +/* + * 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.ml.dlearn.part.local; + +import java.util.List; +import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; +import org.apache.ignite.ml.dlearn.part.LabeledDatasetDLearnPartition; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +/** */ +public class LocalLabeledDatasetDLearnPartitionTransformer + extends LocalDatasetDLearnPartitionTransformer> { + /** */ + private static final long serialVersionUID = -8438445094768312331L; + + /** */ + private final IgniteFunction lbExtractor; + + /** */ + public LocalLabeledDatasetDLearnPartitionTransformer(IgniteFunction extractor, + IgniteFunction lbExtractor) { + super(extractor); + this.lbExtractor = lbExtractor; + } + + /** */ + @SuppressWarnings("unchecked") + @Override public void accept(LocalDLearnPartition oldPart, + LabeledDatasetDLearnPartition newPart) { + super.accept(oldPart, newPart); + List partData = oldPart.getPartData(); + L[] labels = (L[]) new Object[partData.size()]; + for (int i = 0; i < partData.size(); i++) + labels[i] = lbExtractor.apply(partData.get(i)); + newPart.setLabels(labels); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/package-info.java new file mode 100644 index 0000000000000..8935df016e1ca --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Root package for different d-learn partitions. + */ +package org.apache.ignite.ml.dlearn.part; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextPartitionKey.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextPartitionKey.java new file mode 100644 index 0000000000000..8b33f126bdf9d --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextPartitionKey.java @@ -0,0 +1,93 @@ +/* + * 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.ml.dlearn.utils; + +import java.io.Serializable; +import java.util.Objects; +import java.util.UUID; + +/** + * Key used to identify a-learn partition objects. + */ +public class DLearnContextPartitionKey implements Serializable { + /** */ + private static final long serialVersionUID = 9005844909381326835L; + + /** + * Index of partition. + */ + private final int part; + + /** + * Id of learning context. + */ + private final UUID learningCtxId; + + /** + * Key of the object. + */ + private final String key; + + /** */ + public DLearnContextPartitionKey(int part, UUID learningCtxId, String key) { + this.part = part; + this.learningCtxId = learningCtxId; + this.key = key; + } + + /** */ + public int getPart() { + return part; + } + + /** */ + public UUID getLearningCtxId() { + return learningCtxId; + } + + /** */ + public String getKey() { + return key; + } + + /** */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + DLearnContextPartitionKey key1 = (DLearnContextPartitionKey)o; + return part == key1.part && + Objects.equals(learningCtxId, key1.learningCtxId) && + Objects.equals(key, key1.key); + } + + /** */ + @Override public int hashCode() { + return Objects.hash(part, learningCtxId, key); + } + + /** */ + @Override public String toString() { + return "DLearnContextPartitionKey{" + + "part=" + part + + ", learningCtxId=" + learningCtxId + + ", key='" + key + '\'' + + '}'; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/package-info.java new file mode 100644 index 0000000000000..c09fbfea59f5e --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Contains util classes used in d-learn module. + */ +package org.apache.ignite.ml.dlearn.utils; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java new file mode 100644 index 0000000000000..483b0cf3fc0d2 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java @@ -0,0 +1,84 @@ +/* + * 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.ml.regressions.linear; + +import org.apache.ignite.ml.Trainer; +import org.apache.ignite.ml.dlearn.DLearnContext; +import org.apache.ignite.ml.dlearn.part.LabeledDatasetDLearnPartition; + +/** */ +public class LinearRegressionLSQRTrainer implements Trainer> { + /** */ + private static final String A_NAME = "a"; + + /** */ + @Override public LinearRegressionModel train(DLearnContext learningCtx) { + preProcessContext(learningCtx); + +// DistributedLSQR lsqr = new DistributedLSQR<>( +// learningCtx, +// part -> part.get(A_NAME), +// LabeledDatasetLearningContextPartition::getLabels +// ); +// +// LSQRResult res = lsqr.solve(0, 1e-8, 1e-8, 1e8, -1, false, null); +// +// postProcessContext(learningCtx); +// +// double[] x = res.getX(); +// double[] weights = Arrays.copyOfRange(x, 1, x.length); +// double intercept = x[0]; +// +// return new LinearRegressionModel(new DenseLocalOnHeapVector(weights), intercept); + return null; + } + + /** + * Processing of given learning context before training. + */ + private void preProcessContext(DLearnContext learningCtx) { + learningCtx.compute(part -> { + double[] features = part.getFeatures(); + + int rows = part.getRows(); + double[] a = addInterceptCoefficientColumn(features, rows); + +// part.put(A_NAME, a); + }); + } + + /** + * Processing of given learning context after training. + */ + private void postProcessContext(DLearnContext learningCtx) { +// learningCtx.compute(part -> part.remove(A_NAME)); + } + + /** + * Adds intercept coefficient (1.0) columns to the matrix. + * + * @return matrix with intercept coefficient column + */ + private double[] addInterceptCoefficientColumn(double[] features, int rows) { + double[] res = new double[features.length + rows]; + System.arraycopy(features, 0, res, rows, features.length); + for (int i = 0; i < rows; i++) + res[i] = 1.0; + return res; + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java b/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java new file mode 100644 index 0000000000000..25326ca5295d4 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java @@ -0,0 +1,119 @@ +/* + * 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.ml; + +import java.util.Arrays; +import java.util.Scanner; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.ml.dlearn.DLearnContext; +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContextFactory; +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; +import org.apache.ignite.ml.dlearn.dataset.DatasetMathUtils; +import org.apache.ignite.ml.dlearn.part.LabeledDatasetDLearnPartition; +import org.apache.ignite.ml.dlearn.part.cache.CacheLabeledDatasetDLearnPartitionTransformer; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** */ +public class Playground extends GridCommonAbstractTest { + /** Number of nodes in grid */ + private static final int NODE_COUNT = 4; + + /** */ + private Ignite ignite; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + for (int i = 1; i <= NODE_COUNT; i++) + startGrid(i); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() { + stopAllGrids(); + } + + /** + * {@inheritDoc} + */ + @Override protected void beforeTest() throws Exception { + /* Grid instance. */ + ignite = grid(NODE_COUNT); + ignite.configuration().setPeerClassLoadingEnabled(true); + IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); + } + + /** */ + public void testTrainOnBostonDataset() { + IgniteCache bostonDataset = loadDataset(); + + // Initialization of d-learn context, after this step context cache will be created with partitions placed on + // the same nodes as the upstream Ignite Cache (in this case bostonDataset). + DLearnContext> cacheLearningCtx = + new CacheDLearnContextFactory<>(ignite, bostonDataset).createContext(); + + // Loading of the d-learn context. During this step data will be transferred from the upstream cache to context + // cache with specified transformation (it will be performed locally because partitions are on the same nodes). + // In this case for every partition in upstream cache will be created labeled dataset partition and this new + // partition will be filled with help of specified feature and label extractors. + DLearnContext> datasetLearningCtx = cacheLearningCtx + .transform( + new CacheLabeledDatasetDLearnPartitionTransformer<>( + (k, v) -> Arrays.copyOfRange(v, 1, v.length), // specify feature extractor + (k, v) -> v[0] // specify label extractor + ), + LabeledDatasetDLearnPartition::new + ); + + // Calculation of mean value. This calculation will be performed in map-reduce manner. + double[] mean = DatasetMathUtils.mean(datasetLearningCtx, new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); + System.err.println("Mean values : " + Arrays.toString(mean)); + + // Calculation of standard deviation. This calculation will be performed in map-reduce manner. + double[] std = DatasetMathUtils.std(datasetLearningCtx, new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); + System.err.println("Std values : " + Arrays.toString(std)); + } + + /** */ + private IgniteCache loadDataset() { + CacheConfiguration cc = new CacheConfiguration<>(); + cc.setName("BOSTON_DATASET"); + cc.setAffinity(new RendezvousAffinityFunction(true, 10)); + IgniteCache cache = ignite.createCache(cc); + + int nvars = 13; + Scanner scanner = new Scanner(this.getClass().getClassLoader().getResourceAsStream("datasets/regression/boston.csv")); + int i = 0; + while (scanner.hasNextLine()) { + String row = scanner.nextLine(); + int j = 0; + double[] r = new double[nvars + 1]; + for (String feature : row.split(",")) { + r[j] = Double.parseDouble(feature); + j++; + } + cache.put(i, r); + i++; + } + + return cache; + } +} From 104b3bb9badcea4d0d907cca39624cc84ef9544d Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Tue, 16 Jan 2018 23:05:04 +0300 Subject: [PATCH 02/38] IGNITE-7437 Introduce DLearnContextTransformers into ML module. --- .../ignite/ml/dlearn/DLearnContext.java | 13 +++ .../DLearnContextTransformers.java | 48 +++++++++ ...acheDatasetDLearnPartitionTransformer.java | 89 +++++++++++++++++ ...eledDatasetDLearnPartitionTransformer.java | 99 +++++++++++++++++++ ...ocalDatasetDLearnPartitionTransformer.java | 64 ++++++++++++ ...eledDatasetDLearnPartitionTransformer.java | 51 ++++++++++ .../context/transformer/package-info.java | 22 +++++ .../utils/DLearnContextTransformer.java | 29 ++++++ .../java/org/apache/ignite/ml/Playground.java | 10 +- 9 files changed, 420 insertions(+), 5 deletions(-) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextTransformer.java diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java index be80e4c8d9a29..0453898a24dda 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java @@ -17,6 +17,7 @@ package org.apache.ignite.ml.dlearn; +import org.apache.ignite.ml.dlearn.utils.DLearnContextTransformer; import org.apache.ignite.ml.math.functions.IgniteBiConsumer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; @@ -88,4 +89,16 @@ default public void compute(IgniteConsumer

mapper) { * @return new learning context */ public DLearnContext transform(IgniteBiConsumer transformer, DLearnPartitionFactory partFactory); + + /** + * Transforms current learning context into another learning context which contains another type of d-learn + * partitions. Transformation doesn't involve new cache instantiation or network data transfer, it just performs + * {@code #transform(IgniteBiConsumer, DLearnPartitionFactory)} locally on every partition in the current context + * and saves results into the same context cache, but with a new context id. + * + * @param transformer transformer + * @param type of new d-learn partition + * @return new learning context + */ + public DLearnContext transform(DLearnContextTransformer transformer); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java new file mode 100644 index 0000000000000..40dc6146d87f6 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java @@ -0,0 +1,48 @@ +package org.apache.ignite.ml.dlearn.context.transformer; + +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; +import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; +import org.apache.ignite.ml.dlearn.part.DatasetDLeanPartition; +import org.apache.ignite.ml.dlearn.part.LabeledDatasetDLearnPartition; +import org.apache.ignite.ml.dlearn.context.transformer.cache.CacheDatasetDLearnPartitionTransformer; +import org.apache.ignite.ml.dlearn.context.transformer.cache.CacheLabeledDatasetDLearnPartitionTransformer; +import org.apache.ignite.ml.dlearn.context.transformer.local.LocalDatasetDLearnPartitionTransformer; +import org.apache.ignite.ml.dlearn.context.transformer.local.LocalLabeledDatasetDLearnPartitionTransformer; +import org.apache.ignite.ml.dlearn.utils.DLearnContextTransformer; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +/** */ +public class DLearnContextTransformers { + /** */ + public static DLearnContextTransformer, DatasetDLeanPartition> cacheToDataset(IgniteBiFunction featureExtractor) { + return new DLearnContextTransformer<>( + new CacheDatasetDLearnPartitionTransformer<>(featureExtractor), + DatasetDLeanPartition::new + ); + } + + /** */ + public static DLearnContextTransformer, LabeledDatasetDLearnPartition> cacheToLabeledDataset(IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + return new DLearnContextTransformer<>( + new CacheLabeledDatasetDLearnPartitionTransformer<>(featureExtractor, lbExtractor), + LabeledDatasetDLearnPartition::new + ); + } + + /** */ + public static DLearnContextTransformer, DatasetDLeanPartition> localToDataset(IgniteFunction featureExtractor) { + return new DLearnContextTransformer<>( + new LocalDatasetDLearnPartitionTransformer<>(featureExtractor), + DatasetDLeanPartition::new + ); + } + + /** */ + public static DLearnContextTransformer, LabeledDatasetDLearnPartition> localToLabeledDataset(IgniteFunction featureExtractor, IgniteFunction lbExtractor) { + return new DLearnContextTransformer<>( + new LocalLabeledDatasetDLearnPartitionTransformer<>(featureExtractor, lbExtractor), + LabeledDatasetDLearnPartition::new + ); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java new file mode 100644 index 0000000000000..6c1e5812b93e8 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java @@ -0,0 +1,89 @@ +/* + * 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.ml.dlearn.context.transformer.cache; + +import java.util.ArrayList; +import java.util.List; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; +import org.apache.ignite.ml.dlearn.part.DatasetDLeanPartition; +import org.apache.ignite.ml.math.functions.IgniteBiConsumer; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +/** */ +public class CacheDatasetDLearnPartitionTransformer implements IgniteBiConsumer, DatasetDLeanPartition> { + /** */ + private static final long serialVersionUID = -7398727071330763144L; + + /** */ + private final IgniteBiFunction featureExtractor; + + /** */ + public CacheDatasetDLearnPartitionTransformer(IgniteBiFunction featureExtractor) { + this.featureExtractor = featureExtractor; + } + + /** */ + @Override public void accept(CacheDLearnPartition oldPart, DatasetDLeanPartition newPart) { + List> partData = queryPartDataIntoList(oldPart); + + double[] features = null; + int m = partData.size(), n = 0; + for (int i = 0; i < partData.size(); i++) { + Cache.Entry entry = partData.get(i); + double[] rowFeatures = featureExtractor.apply(entry.getKey(), entry.getValue()); + + if (i == 0) { + n = rowFeatures.length; + features = new double[m * n]; + } + + if (rowFeatures.length != n) + throw new IllegalStateException(); + + for (int j = 0; j < rowFeatures.length; j++) + features[j * m + i] = rowFeatures[j]; + } + newPart.setFeatures(features); + newPart.setRows(m); + } + + /** */ + private List> queryPartDataIntoList(CacheDLearnPartition oldPart) { + List> partData = new ArrayList<>(); + for (Cache.Entry entry : queryPartData(oldPart)) + partData.add(entry); + return partData; + } + + /** */ + private Iterable> queryPartData(CacheDLearnPartition oldPart) { + Ignite ignite = Ignition.localIgnite(); + IgniteCache upstreamCache = ignite.cache(oldPart.getUpstreamCacheName()); + + ScanQuery qry = new ScanQuery<>(); + qry.setLocal(true); + qry.setPartition(oldPart.getPart()); + + return upstreamCache.query(qry); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java new file mode 100644 index 0000000000000..37af4dc4625ad --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java @@ -0,0 +1,99 @@ +/* + * 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.ml.dlearn.context.transformer.cache; + +import java.util.ArrayList; +import java.util.List; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; +import org.apache.ignite.ml.dlearn.part.LabeledDatasetDLearnPartition; +import org.apache.ignite.ml.math.functions.IgniteBiConsumer; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +/** */ +public class CacheLabeledDatasetDLearnPartitionTransformer implements IgniteBiConsumer, LabeledDatasetDLearnPartition> { + /** */ + private static final long serialVersionUID = 3479218902890029731L; + + /** */ + private final IgniteBiFunction featureExtractor; + + /** */ + private final IgniteBiFunction lbExtractor; + + /** */ + public CacheLabeledDatasetDLearnPartitionTransformer( + IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + this.featureExtractor = featureExtractor; + this.lbExtractor = lbExtractor; + } + + /** */ + @SuppressWarnings("unchecked") + @Override public void accept(CacheDLearnPartition oldPart, LabeledDatasetDLearnPartition newPart) { + List> partData = queryPartDataIntoList(oldPart); + + int m = partData.size(), n = 0; + double[] features = null; + L[] labels = (L[]) new Object[m]; + for (int i = 0; i < partData.size(); i++) { + Cache.Entry entry = partData.get(i); + double[] rowFeatures = featureExtractor.apply(entry.getKey(), entry.getValue()); + labels[i] = lbExtractor.apply(entry.getKey(), entry.getValue()); + + if (i == 0) { + n = rowFeatures.length; + features = new double[m * n]; + } + + if (rowFeatures.length != n) + throw new IllegalStateException(); + + for (int j = 0; j < rowFeatures.length; j++) + features[j * m + i] = rowFeatures[j]; + } + newPart.setFeatures(features); + newPart.setRows(m); + newPart.setLabels(labels); + } + + /** */ + private List> queryPartDataIntoList(CacheDLearnPartition oldPart) { + List> partData = new ArrayList<>(); + for (Cache.Entry entry : queryPartData(oldPart)) + partData.add(entry); + return partData; + } + + /** */ + private Iterable> queryPartData(CacheDLearnPartition oldPart) { + Ignite ignite = Ignition.localIgnite(); + IgniteCache upstreamCache = ignite.cache(oldPart.getUpstreamCacheName()); + + ScanQuery qry = new ScanQuery<>(); + qry.setLocal(true); + qry.setPartition(oldPart.getPart()); + + return upstreamCache.query(qry); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java new file mode 100644 index 0000000000000..7cb59be179bd7 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java @@ -0,0 +1,64 @@ +/* + * 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.ml.dlearn.context.transformer.local; + +import java.util.List; +import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; +import org.apache.ignite.ml.dlearn.part.DatasetDLeanPartition; +import org.apache.ignite.ml.math.functions.IgniteBiConsumer; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +/** */ +public class LocalDatasetDLearnPartitionTransformer + implements IgniteBiConsumer, P> { + /** */ + private static final long serialVersionUID = -7567051002880704559L; + + /** */ + private final IgniteFunction featureExtractor; + + /** */ + public LocalDatasetDLearnPartitionTransformer(IgniteFunction featureExtractor) { + this.featureExtractor = featureExtractor; + } + + /** */ + @Override public void accept(LocalDLearnPartition oldPart, P newPart) { + List partData = oldPart.getPartData(); + if (partData != null && !partData.isEmpty()) { + double[] features = null; + int m = partData.size(), n = 0; + for (int i = 0; i < partData.size(); i++) { + double[] rowFeatures = featureExtractor.apply(partData.get(i)); + + if (i == 0) { + n = rowFeatures.length; + features = new double[m * n]; + } + + if (rowFeatures.length != n) + throw new IllegalStateException(); + + for (int j = 0; j < rowFeatures.length; j++) + features[j * m + i] = rowFeatures[j]; + } + newPart.setFeatures(features); + newPart.setRows(m); + } + } +} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java new file mode 100644 index 0000000000000..7412dfb50a979 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java @@ -0,0 +1,51 @@ +/* + * 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.ml.dlearn.context.transformer.local; + +import java.util.List; +import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; +import org.apache.ignite.ml.dlearn.part.LabeledDatasetDLearnPartition; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +/** */ +public class LocalLabeledDatasetDLearnPartitionTransformer + extends LocalDatasetDLearnPartitionTransformer> { + /** */ + private static final long serialVersionUID = -8438445094768312331L; + + /** */ + private final IgniteFunction lbExtractor; + + /** */ + public LocalLabeledDatasetDLearnPartitionTransformer(IgniteFunction featureExtractor, IgniteFunction lbExtractor) { + super(featureExtractor); + this.lbExtractor = lbExtractor; + } + + /** */ + @SuppressWarnings("unchecked") + @Override public void accept(LocalDLearnPartition oldPart, + LabeledDatasetDLearnPartition newPart) { + super.accept(oldPart, newPart); + List partData = oldPart.getPartData(); + L[] labels = (L[]) new Object[partData.size()]; + for (int i = 0; i < partData.size(); i++) + labels[i] = lbExtractor.apply(partData.get(i)); + newPart.setLabels(labels); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/package-info.java new file mode 100644 index 0000000000000..627432fd801c6 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Root package for d-learn (distributed learning) framework. + */ +package org.apache.ignite.ml.dlearn.context.transformer; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextTransformer.java new file mode 100644 index 0000000000000..ba208e28d7856 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextTransformer.java @@ -0,0 +1,29 @@ +package org.apache.ignite.ml.dlearn.utils; + +import org.apache.ignite.ml.dlearn.DLearnPartitionFactory; +import org.apache.ignite.ml.math.functions.IgniteBiConsumer; + +/** */ +public class DLearnContextTransformer { + /** */ + private final IgniteBiConsumer transformer; + + /** */ + private final DLearnPartitionFactory partFactory; + + /** */ + public DLearnContextTransformer(IgniteBiConsumer transformer, DLearnPartitionFactory partFactory) { + this.transformer = transformer; + this.partFactory = partFactory; + } + + /** */ + public IgniteBiConsumer getTransformer() { + return transformer; + } + + /** */ + public DLearnPartitionFactory getPartFactory() { + return partFactory; + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java b/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java index 25326ca5295d4..bd7c41d12e73b 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java @@ -29,7 +29,7 @@ import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; import org.apache.ignite.ml.dlearn.dataset.DatasetMathUtils; import org.apache.ignite.ml.dlearn.part.LabeledDatasetDLearnPartition; -import org.apache.ignite.ml.dlearn.part.cache.CacheLabeledDatasetDLearnPartitionTransformer; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformers; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; /** */ @@ -74,13 +74,13 @@ public void testTrainOnBostonDataset() { // cache with specified transformation (it will be performed locally because partitions are on the same nodes). // In this case for every partition in upstream cache will be created labeled dataset partition and this new // partition will be filled with help of specified feature and label extractors. + DLearnContext> datasetLearningCtx = cacheLearningCtx .transform( - new CacheLabeledDatasetDLearnPartitionTransformer<>( + DLearnContextTransformers.cacheToLabeledDataset( (k, v) -> Arrays.copyOfRange(v, 1, v.length), // specify feature extractor - (k, v) -> v[0] // specify label extractor - ), - LabeledDatasetDLearnPartition::new + (k, v) -> v[0] // specify label extractor + ) ); // Calculation of mean value. This calculation will be performed in map-reduce manner. From 006b749e4742698f837582b3224a93468c88511e Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Tue, 16 Jan 2018 23:18:39 +0300 Subject: [PATCH 03/38] IGNITE-7437 Introduce DLearnContextTransformers into ML module (fix compilation issues). --- .../context/cache/CacheDLearnContext.java | 6 ++ .../context/local/LocalDLearnContext.java | 6 ++ ...acheDatasetDLearnPartitionTransformer.java | 89 ----------------- ...eledDatasetDLearnPartitionTransformer.java | 99 ------------------- ...ocalDatasetDLearnPartitionTransformer.java | 64 ------------ ...eledDatasetDLearnPartitionTransformer.java | 52 ---------- 6 files changed, 12 insertions(+), 304 deletions(-) delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/cache/CacheDatasetDLearnPartitionTransformer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/cache/CacheLabeledDatasetDLearnPartitionTransformer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/local/LocalDatasetDLearnPartitionTransformer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/local/LocalLabeledDatasetDLearnPartitionTransformer.java diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java index aa1b1d5742524..9a17a21103bf8 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java @@ -28,6 +28,7 @@ import org.apache.ignite.ml.dlearn.DLearnPartitionFactory; import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; +import org.apache.ignite.ml.dlearn.utils.DLearnContextTransformer; import org.apache.ignite.ml.math.functions.IgniteBiConsumer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; @@ -115,6 +116,11 @@ public R compute(IgniteBiFunction mapper, IgniteBinaryOperato return new CacheDLearnContext<>(ignite, learningCtxCacheName, partFactory, newLearningCtxId); } + /** */ + @Override public DLearnContext transform(DLearnContextTransformer transformer) { + return transform(transformer.getTransformer(), transformer.getPartFactory()); + } + /** */ private R reduce(Collection results, IgniteBinaryOperator reducer) { R res = null; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java index b36510aed61bd..9a94dc1d7b176 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java @@ -23,6 +23,7 @@ import org.apache.ignite.ml.dlearn.DLearnPartitionFactory; import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; +import org.apache.ignite.ml.dlearn.utils.DLearnContextTransformer; import org.apache.ignite.ml.math.functions.IgniteBiConsumer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; @@ -88,4 +89,9 @@ public LocalDLearnContext(Map learningCtxMap, return new LocalDLearnContext<>(learningCtxMap, partFactory, newLearningCtxId, partitions); } + + /** */ + @Override public DLearnContext transform(DLearnContextTransformer transformer) { + return transform(transformer.getTransformer(), transformer.getPartFactory()); + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/cache/CacheDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/cache/CacheDatasetDLearnPartitionTransformer.java deleted file mode 100644 index 5e134c9c6fbd0..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/cache/CacheDatasetDLearnPartitionTransformer.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * 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.ml.dlearn.part.cache; - -import java.util.ArrayList; -import java.util.List; -import javax.cache.Cache; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.Ignition; -import org.apache.ignite.cache.query.ScanQuery; -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; -import org.apache.ignite.ml.dlearn.part.DatasetDLeanPartition; -import org.apache.ignite.ml.math.functions.IgniteBiConsumer; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; - -/** */ -public class CacheDatasetDLearnPartitionTransformer implements IgniteBiConsumer,DatasetDLeanPartition> { - /** */ - private static final long serialVersionUID = -7398727071330763144L; - - /** */ - private final IgniteBiFunction featureExtractor; - - /** */ - public CacheDatasetDLearnPartitionTransformer(IgniteBiFunction featureExtractor) { - this.featureExtractor = featureExtractor; - } - - /** */ - @Override public void accept(CacheDLearnPartition oldPart, DatasetDLeanPartition newPart) { - List> partData = queryPartDataIntoList(oldPart); - - double[] features = null; - int m = partData.size(), n = 0; - for (int i = 0; i < partData.size(); i++) { - Cache.Entry entry = partData.get(i); - double[] rowFeatures = featureExtractor.apply(entry.getKey(), entry.getValue()); - - if (i == 0) { - n = rowFeatures.length; - features = new double[m * n]; - } - - if (rowFeatures.length != n) - throw new IllegalStateException(); - - for (int j = 0; j < rowFeatures.length; j++) - features[j * m + i] = rowFeatures[j]; - } - newPart.setFeatures(features); - newPart.setRows(m); - } - - /** */ - private List> queryPartDataIntoList(CacheDLearnPartition oldPart) { - List> partData = new ArrayList<>(); - for (Cache.Entry entry : queryPartData(oldPart)) - partData.add(entry); - return partData; - } - - /** */ - private Iterable> queryPartData(CacheDLearnPartition oldPart) { - Ignite ignite = Ignition.localIgnite(); - IgniteCache upstreamCache = ignite.cache(oldPart.getUpstreamCacheName()); - - ScanQuery qry = new ScanQuery<>(); - qry.setLocal(true); - qry.setPartition(oldPart.getPart()); - - return upstreamCache.query(qry); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/cache/CacheLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/cache/CacheLabeledDatasetDLearnPartitionTransformer.java deleted file mode 100644 index 2a893664f7f4f..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/cache/CacheLabeledDatasetDLearnPartitionTransformer.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * 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.ml.dlearn.part.cache; - -import java.util.ArrayList; -import java.util.List; -import javax.cache.Cache; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.Ignition; -import org.apache.ignite.cache.query.ScanQuery; -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; -import org.apache.ignite.ml.dlearn.part.LabeledDatasetDLearnPartition; -import org.apache.ignite.ml.math.functions.IgniteBiConsumer; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; - -/** */ -public class CacheLabeledDatasetDLearnPartitionTransformer implements IgniteBiConsumer, LabeledDatasetDLearnPartition> { - /** */ - private static final long serialVersionUID = 3479218902890029731L; - - /** */ - private final IgniteBiFunction featureExtractor; - - /** */ - private final IgniteBiFunction lbExtractor; - - /** */ - public CacheLabeledDatasetDLearnPartitionTransformer( - IgniteBiFunction featureExtractor, - IgniteBiFunction lbExtractor) { - this.featureExtractor = featureExtractor; - this.lbExtractor = lbExtractor; - } - - /** */ - @SuppressWarnings("unchecked") - @Override public void accept(CacheDLearnPartition oldPart, LabeledDatasetDLearnPartition newPart) { - List> partData = queryPartDataIntoList(oldPart); - - int m = partData.size(), n = 0; - double[] features = null; - L[] labels = (L[]) new Object[m]; - for (int i = 0; i < partData.size(); i++) { - Cache.Entry entry = partData.get(i); - double[] rowFeatures = featureExtractor.apply(entry.getKey(), entry.getValue()); - labels[i] = lbExtractor.apply(entry.getKey(), entry.getValue()); - - if (i == 0) { - n = rowFeatures.length; - features = new double[m * n]; - } - - if (rowFeatures.length != n) - throw new IllegalStateException(); - - for (int j = 0; j < rowFeatures.length; j++) - features[j * m + i] = rowFeatures[j]; - } - newPart.setFeatures(features); - newPart.setRows(m); - newPart.setLabels(labels); - } - - /** */ - private List> queryPartDataIntoList(CacheDLearnPartition oldPart) { - List> partData = new ArrayList<>(); - for (Cache.Entry entry : queryPartData(oldPart)) - partData.add(entry); - return partData; - } - - /** */ - private Iterable> queryPartData(CacheDLearnPartition oldPart) { - Ignite ignite = Ignition.localIgnite(); - IgniteCache upstreamCache = ignite.cache(oldPart.getUpstreamCacheName()); - - ScanQuery qry = new ScanQuery<>(); - qry.setLocal(true); - qry.setPartition(oldPart.getPart()); - - return upstreamCache.query(qry); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/local/LocalDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/local/LocalDatasetDLearnPartitionTransformer.java deleted file mode 100644 index bb178364d1170..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/local/LocalDatasetDLearnPartitionTransformer.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * 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.ml.dlearn.part.local; - -import java.util.List; -import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; -import org.apache.ignite.ml.dlearn.part.DatasetDLeanPartition; -import org.apache.ignite.ml.math.functions.IgniteBiConsumer; -import org.apache.ignite.ml.math.functions.IgniteFunction; - -/** */ -public class LocalDatasetDLearnPartitionTransformer - implements IgniteBiConsumer, P> { - /** */ - private static final long serialVersionUID = -7567051002880704559L; - - /** */ - private final IgniteFunction featureExtractor; - - /** */ - public LocalDatasetDLearnPartitionTransformer(IgniteFunction extractor) { - featureExtractor = extractor; - } - - /** */ - @Override public void accept(LocalDLearnPartition oldPart, P newPart) { - List partData = oldPart.getPartData(); - if (partData != null && !partData.isEmpty()) { - double[] features = null; - int m = partData.size(), n = 0; - for (int i = 0; i < partData.size(); i++) { - double[] rowFeatures = featureExtractor.apply(partData.get(i)); - - if (i == 0) { - n = rowFeatures.length; - features = new double[m * n]; - } - - if (rowFeatures.length != n) - throw new IllegalStateException(); - - for (int j = 0; j < rowFeatures.length; j++) - features[j * m + i] = rowFeatures[j]; - } - newPart.setFeatures(features); - newPart.setRows(m); - } - } -} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/local/LocalLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/local/LocalLabeledDatasetDLearnPartitionTransformer.java deleted file mode 100644 index 89d799f2096ff..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/local/LocalLabeledDatasetDLearnPartitionTransformer.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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.ml.dlearn.part.local; - -import java.util.List; -import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; -import org.apache.ignite.ml.dlearn.part.LabeledDatasetDLearnPartition; -import org.apache.ignite.ml.math.functions.IgniteFunction; - -/** */ -public class LocalLabeledDatasetDLearnPartitionTransformer - extends LocalDatasetDLearnPartitionTransformer> { - /** */ - private static final long serialVersionUID = -8438445094768312331L; - - /** */ - private final IgniteFunction lbExtractor; - - /** */ - public LocalLabeledDatasetDLearnPartitionTransformer(IgniteFunction extractor, - IgniteFunction lbExtractor) { - super(extractor); - this.lbExtractor = lbExtractor; - } - - /** */ - @SuppressWarnings("unchecked") - @Override public void accept(LocalDLearnPartition oldPart, - LabeledDatasetDLearnPartition newPart) { - super.accept(oldPart, newPart); - List partData = oldPart.getPartData(); - L[] labels = (L[]) new Object[partData.size()]; - for (int i = 0; i < partData.size(); i++) - labels[i] = lbExtractor.apply(partData.get(i)); - newPart.setLabels(labels); - } -} From d8696b6210dccaa92f4de459a9590f9e0896c7db Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Wed, 17 Jan 2018 11:26:52 +0300 Subject: [PATCH 04/38] IGNITE-7437 Move dataset API into d-learn context wrapper to avoid introducing an additional util classes. --- .../ignite/ml/dlearn/DLearnContext.java | 17 +--- .../context/cache/CacheDLearnContext.java | 15 ++-- .../context/local/LocalDLearnContext.java | 16 ++-- .../transformer/DLearnContextTransformer.java | 13 +++ .../DLearnContextTransformers.java | 33 ++------ ...acheDatasetDLearnPartitionTransformer.java | 21 ++++- ...eledDatasetDLearnPartitionTransformer.java | 21 ++++- ...ocalDatasetDLearnPartitionTransformer.java | 23 ++++-- ...eledDatasetDLearnPartitionTransformer.java | 56 ++++++++++--- .../dataset/AbstractDLearnContextWrapper.java | 50 ++++++++++++ ...tasetMathUtils.java => DLearnDataset.java} | 79 ++++++++++++++++--- .../dlearn/dataset/DLearnLabeledDataset.java | 29 +++++++ .../part/DLeanDatasetPartition.java} | 6 +- .../part/DLearnLabeledDatasetPartition.java} | 6 +- .../{ => dataset}/part/package-info.java | 2 +- .../utils/DLearnContextTransformer.java | 29 ------- .../linear/LinearRegressionLSQRTrainer.java | 10 +-- .../java/org/apache/ignite/ml/Playground.java | 10 +-- 18 files changed, 295 insertions(+), 141 deletions(-) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java rename modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/{DatasetMathUtils.java => DLearnDataset.java} (50%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnLabeledDataset.java rename modules/ml/src/main/java/org/apache/ignite/ml/dlearn/{part/DatasetDLeanPartition.java => dataset/part/DLeanDatasetPartition.java} (92%) rename modules/ml/src/main/java/org/apache/ignite/ml/dlearn/{part/LabeledDatasetDLearnPartition.java => dataset/part/DLearnLabeledDatasetPartition.java} (88%) rename modules/ml/src/main/java/org/apache/ignite/ml/dlearn/{ => dataset}/part/package-info.java (94%) delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextTransformer.java diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java index 0453898a24dda..5ea9dff111d10 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java @@ -17,7 +17,7 @@ package org.apache.ignite.ml.dlearn; -import org.apache.ignite.ml.dlearn.utils.DLearnContextTransformer; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; import org.apache.ignite.ml.math.functions.IgniteBiConsumer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; @@ -77,19 +77,6 @@ default public void compute(IgniteConsumer

mapper) { compute((part, partIdx) -> mapper.accept(part)); } - /** - * Transforms current learning context into another learning context which contains another type of d-learn - * partitions. Transformation doesn't involve new cache instantiation or network data transfer, it just performs - * {@code #transform(IgniteBiConsumer, DLearnPartitionFactory)} locally on every partition in the current context - * and saves results into the same context cache, but with a new context id. - * - * @param transformer transformer function which makes the new d-learn partition from the old one - * @param partFactory d-learn partition factory - * @param type of the new d-learn partition - * @return new learning context - */ - public DLearnContext transform(IgniteBiConsumer transformer, DLearnPartitionFactory partFactory); - /** * Transforms current learning context into another learning context which contains another type of d-learn * partitions. Transformation doesn't involve new cache instantiation or network data transfer, it just performs @@ -100,5 +87,5 @@ default public void compute(IgniteConsumer

mapper) { * @param type of new d-learn partition * @return new learning context */ - public DLearnContext transform(DLearnContextTransformer transformer); + public > C transform(DLearnContextTransformer transformer); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java index 9a17a21103bf8..d13f385617eab 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java @@ -28,7 +28,7 @@ import org.apache.ignite.ml.dlearn.DLearnPartitionFactory; import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; -import org.apache.ignite.ml.dlearn.utils.DLearnContextTransformer; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; import org.apache.ignite.ml.math.functions.IgniteBiConsumer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; @@ -103,22 +103,19 @@ public R compute(IgniteBiFunction mapper, IgniteBinaryOperato } /** */ - @Override public DLearnContext transform(IgniteBiConsumer transformer, DLearnPartitionFactory partFactory) { + @Override public > C transform(DLearnContextTransformer transformer) { UUID newLearningCtxId = UUID.randomUUID(); compute((part, partIdx) -> { IgniteCache learningCtxCache = ignite.cache(learningCtxCacheName); DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, newLearningCtxId, partIdx); - T newPart = partFactory.createPartition(storage); - transformer.accept(part, newPart); + T newPart = transformer.createPartition(storage); + transformer.transform(part, newPart); }); - return new CacheDLearnContext<>(ignite, learningCtxCacheName, partFactory, newLearningCtxId); - } + DLearnContext newCtx = new CacheDLearnContext<>(ignite, learningCtxCacheName, transformer, newLearningCtxId); - /** */ - @Override public DLearnContext transform(DLearnContextTransformer transformer) { - return transform(transformer.getTransformer(), transformer.getPartFactory()); + return transformer.wrapContext(newCtx); } /** */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java index 9a94dc1d7b176..a40ba2f50774a 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java @@ -23,7 +23,7 @@ import org.apache.ignite.ml.dlearn.DLearnPartitionFactory; import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; -import org.apache.ignite.ml.dlearn.utils.DLearnContextTransformer; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; import org.apache.ignite.ml.math.functions.IgniteBiConsumer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; @@ -77,21 +77,17 @@ public LocalDLearnContext(Map learningCtxMap, } /** */ - @Override public DLearnContext transform(IgniteBiConsumer transformer, - DLearnPartitionFactory partFactory) { + @Override public > C transform(DLearnContextTransformer transformer) { UUID newLearningCtxId = UUID.randomUUID(); compute((part, partIdx) -> { DLearnPartitionStorage newStorage = new LocalDLearnPartitionStorage(learningCtxMap, newLearningCtxId, partIdx); - T newPart = partFactory.createPartition(newStorage); - transformer.accept(part, newPart); + T newPart = transformer.createPartition(newStorage); + transformer.transform(part, newPart); }); - return new LocalDLearnContext<>(learningCtxMap, partFactory, newLearningCtxId, partitions); - } + DLearnContext newCtx = new LocalDLearnContext<>(learningCtxMap, transformer, newLearningCtxId, partitions); - /** */ - @Override public DLearnContext transform(DLearnContextTransformer transformer) { - return transform(transformer.getTransformer(), transformer.getPartFactory()); + return transformer.wrapContext(newCtx); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformer.java new file mode 100644 index 0000000000000..6f7263f88a8c2 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformer.java @@ -0,0 +1,13 @@ +package org.apache.ignite.ml.dlearn.context.transformer; + +import org.apache.ignite.ml.dlearn.DLearnContext; +import org.apache.ignite.ml.dlearn.DLearnPartitionFactory; + +/** */ +public interface DLearnContextTransformer> extends DLearnPartitionFactory { + /** */ + public void transform(P oldPart, T newPart); + + /** */ + public C wrapContext(DLearnContext ctx); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java index 40dc6146d87f6..ed5463c386bc0 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java @@ -1,48 +1,31 @@ package org.apache.ignite.ml.dlearn.context.transformer; -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; -import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; -import org.apache.ignite.ml.dlearn.part.DatasetDLeanPartition; -import org.apache.ignite.ml.dlearn.part.LabeledDatasetDLearnPartition; import org.apache.ignite.ml.dlearn.context.transformer.cache.CacheDatasetDLearnPartitionTransformer; import org.apache.ignite.ml.dlearn.context.transformer.cache.CacheLabeledDatasetDLearnPartitionTransformer; import org.apache.ignite.ml.dlearn.context.transformer.local.LocalDatasetDLearnPartitionTransformer; import org.apache.ignite.ml.dlearn.context.transformer.local.LocalLabeledDatasetDLearnPartitionTransformer; -import org.apache.ignite.ml.dlearn.utils.DLearnContextTransformer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.functions.IgniteFunction; /** */ public class DLearnContextTransformers { /** */ - public static DLearnContextTransformer, DatasetDLeanPartition> cacheToDataset(IgniteBiFunction featureExtractor) { - return new DLearnContextTransformer<>( - new CacheDatasetDLearnPartitionTransformer<>(featureExtractor), - DatasetDLeanPartition::new - ); + public static CacheDatasetDLearnPartitionTransformer cacheToDataset(IgniteBiFunction featureExtractor) { + return new CacheDatasetDLearnPartitionTransformer<>(featureExtractor); } /** */ - public static DLearnContextTransformer, LabeledDatasetDLearnPartition> cacheToLabeledDataset(IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { - return new DLearnContextTransformer<>( - new CacheLabeledDatasetDLearnPartitionTransformer<>(featureExtractor, lbExtractor), - LabeledDatasetDLearnPartition::new - ); + public static CacheLabeledDatasetDLearnPartitionTransformer cacheToLabeledDataset(IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + return new CacheLabeledDatasetDLearnPartitionTransformer<>(featureExtractor, lbExtractor); } /** */ - public static DLearnContextTransformer, DatasetDLeanPartition> localToDataset(IgniteFunction featureExtractor) { - return new DLearnContextTransformer<>( - new LocalDatasetDLearnPartitionTransformer<>(featureExtractor), - DatasetDLeanPartition::new - ); + public static LocalDatasetDLearnPartitionTransformer localToDataset(IgniteFunction featureExtractor) { + return new LocalDatasetDLearnPartitionTransformer<>(featureExtractor); } /** */ - public static DLearnContextTransformer, LabeledDatasetDLearnPartition> localToLabeledDataset(IgniteFunction featureExtractor, IgniteFunction lbExtractor) { - return new DLearnContextTransformer<>( - new LocalLabeledDatasetDLearnPartitionTransformer<>(featureExtractor, lbExtractor), - LabeledDatasetDLearnPartition::new - ); + public static LocalLabeledDatasetDLearnPartitionTransformer localToLabeledDataset(IgniteFunction featureExtractor, IgniteFunction lbExtractor) { + return new LocalLabeledDatasetDLearnPartitionTransformer<>(featureExtractor, lbExtractor); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java index 6c1e5812b93e8..7a57084295f4e 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java @@ -24,13 +24,16 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.Ignition; import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.ml.dlearn.DLearnContext; +import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; -import org.apache.ignite.ml.dlearn.part.DatasetDLeanPartition; -import org.apache.ignite.ml.math.functions.IgniteBiConsumer; +import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; +import org.apache.ignite.ml.dlearn.dataset.part.DLeanDatasetPartition; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; /** */ -public class CacheDatasetDLearnPartitionTransformer implements IgniteBiConsumer, DatasetDLeanPartition> { +public class CacheDatasetDLearnPartitionTransformer implements DLearnContextTransformer, DLeanDatasetPartition, DLearnDataset> { /** */ private static final long serialVersionUID = -7398727071330763144L; @@ -43,7 +46,7 @@ public CacheDatasetDLearnPartitionTransformer(IgniteBiFunction f } /** */ - @Override public void accept(CacheDLearnPartition oldPart, DatasetDLeanPartition newPart) { + @Override public void transform(CacheDLearnPartition oldPart, DLeanDatasetPartition newPart) { List> partData = queryPartDataIntoList(oldPart); double[] features = null; @@ -67,6 +70,16 @@ public CacheDatasetDLearnPartitionTransformer(IgniteBiFunction f newPart.setRows(m); } + /** */ + @Override public DLearnDataset wrapContext(DLearnContext ctx) { + return new DLearnDataset<>(ctx); + } + + /** */ + @Override public DLeanDatasetPartition createPartition(DLearnPartitionStorage storage) { + return new DLeanDatasetPartition(storage); + } + /** */ private List> queryPartDataIntoList(CacheDLearnPartition oldPart) { List> partData = new ArrayList<>(); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java index 37af4dc4625ad..83a271094d700 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java @@ -24,13 +24,16 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.Ignition; import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.ml.dlearn.DLearnContext; +import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; -import org.apache.ignite.ml.dlearn.part.LabeledDatasetDLearnPartition; -import org.apache.ignite.ml.math.functions.IgniteBiConsumer; +import org.apache.ignite.ml.dlearn.dataset.DLearnLabeledDataset; +import org.apache.ignite.ml.dlearn.dataset.part.DLearnLabeledDatasetPartition; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; /** */ -public class CacheLabeledDatasetDLearnPartitionTransformer implements IgniteBiConsumer, LabeledDatasetDLearnPartition> { +public class CacheLabeledDatasetDLearnPartitionTransformer implements DLearnContextTransformer, DLearnLabeledDatasetPartition, DLearnLabeledDataset> { /** */ private static final long serialVersionUID = 3479218902890029731L; @@ -50,7 +53,7 @@ public CacheLabeledDatasetDLearnPartitionTransformer( /** */ @SuppressWarnings("unchecked") - @Override public void accept(CacheDLearnPartition oldPart, LabeledDatasetDLearnPartition newPart) { + @Override public void transform(CacheDLearnPartition oldPart, DLearnLabeledDatasetPartition newPart) { List> partData = queryPartDataIntoList(oldPart); int m = partData.size(), n = 0; @@ -77,6 +80,16 @@ public CacheLabeledDatasetDLearnPartitionTransformer( newPart.setLabels(labels); } + /** */ + @Override public DLearnLabeledDataset wrapContext(DLearnContext> ctx) { + return new DLearnLabeledDataset<>(ctx); + } + + /** */ + @Override public DLearnLabeledDatasetPartition createPartition(DLearnPartitionStorage storage) { + return new DLearnLabeledDatasetPartition<>(storage); + } + /** */ private List> queryPartDataIntoList(CacheDLearnPartition oldPart) { List> partData = new ArrayList<>(); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java index 7cb59be179bd7..ff16fa980ed46 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java @@ -18,14 +18,17 @@ package org.apache.ignite.ml.dlearn.context.transformer.local; import java.util.List; +import org.apache.ignite.ml.dlearn.DLearnContext; +import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; -import org.apache.ignite.ml.dlearn.part.DatasetDLeanPartition; -import org.apache.ignite.ml.math.functions.IgniteBiConsumer; +import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; +import org.apache.ignite.ml.dlearn.dataset.part.DLeanDatasetPartition; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; import org.apache.ignite.ml.math.functions.IgniteFunction; /** */ -public class LocalDatasetDLearnPartitionTransformer - implements IgniteBiConsumer, P> { +public class LocalDatasetDLearnPartitionTransformer + implements DLearnContextTransformer, DLeanDatasetPartition, DLearnDataset> { /** */ private static final long serialVersionUID = -7567051002880704559L; @@ -38,7 +41,7 @@ public LocalDatasetDLearnPartitionTransformer(IgniteFunction featur } /** */ - @Override public void accept(LocalDLearnPartition oldPart, P newPart) { + @Override public void transform(LocalDLearnPartition oldPart, DLeanDatasetPartition newPart) { List partData = oldPart.getPartData(); if (partData != null && !partData.isEmpty()) { double[] features = null; @@ -61,4 +64,14 @@ public LocalDatasetDLearnPartitionTransformer(IgniteFunction featur newPart.setRows(m); } } + + /** */ + @Override public DLearnDataset wrapContext(DLearnContext ctx) { + return new DLearnDataset<>(ctx); + } + + /** */ + @Override public DLeanDatasetPartition createPartition(DLearnPartitionStorage storage) { + return new DLeanDatasetPartition(storage); + } } \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java index 7412dfb50a979..054c4dcd82029 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java @@ -18,34 +18,70 @@ package org.apache.ignite.ml.dlearn.context.transformer.local; import java.util.List; +import org.apache.ignite.ml.dlearn.DLearnContext; +import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; -import org.apache.ignite.ml.dlearn.part.LabeledDatasetDLearnPartition; +import org.apache.ignite.ml.dlearn.dataset.DLearnLabeledDataset; +import org.apache.ignite.ml.dlearn.dataset.part.DLearnLabeledDatasetPartition; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; import org.apache.ignite.ml.math.functions.IgniteFunction; /** */ public class LocalLabeledDatasetDLearnPartitionTransformer - extends LocalDatasetDLearnPartitionTransformer> { + implements DLearnContextTransformer,DLearnLabeledDatasetPartition, DLearnLabeledDataset> { /** */ private static final long serialVersionUID = -8438445094768312331L; + /** */ + private final IgniteFunction featureExtractor; + /** */ private final IgniteFunction lbExtractor; /** */ public LocalLabeledDatasetDLearnPartitionTransformer(IgniteFunction featureExtractor, IgniteFunction lbExtractor) { - super(featureExtractor); + this.featureExtractor = featureExtractor; this.lbExtractor = lbExtractor; } /** */ @SuppressWarnings("unchecked") - @Override public void accept(LocalDLearnPartition oldPart, - LabeledDatasetDLearnPartition newPart) { - super.accept(oldPart, newPart); + @Override public void transform(LocalDLearnPartition oldPart, DLearnLabeledDatasetPartition newPart) { List partData = oldPart.getPartData(); - L[] labels = (L[]) new Object[partData.size()]; - for (int i = 0; i < partData.size(); i++) - labels[i] = lbExtractor.apply(partData.get(i)); - newPart.setLabels(labels); + if (partData != null && !partData.isEmpty()) { + double[] features = null; + int m = partData.size(), n = 0; + for (int i = 0; i < partData.size(); i++) { + double[] rowFeatures = featureExtractor.apply(partData.get(i)); + + if (i == 0) { + n = rowFeatures.length; + features = new double[m * n]; + } + + if (rowFeatures.length != n) + throw new IllegalStateException(); + + for (int j = 0; j < rowFeatures.length; j++) + features[j * m + i] = rowFeatures[j]; + } + newPart.setFeatures(features); + newPart.setRows(m); + + L[] labels = (L[]) new Object[partData.size()]; + for (int i = 0; i < partData.size(); i++) + labels[i] = lbExtractor.apply(partData.get(i)); + newPart.setLabels(labels); + } + } + + /** */ + @Override public DLearnLabeledDataset wrapContext(DLearnContext> ctx) { + return new DLearnLabeledDataset<>(ctx); + } + + /** */ + @Override public DLearnLabeledDatasetPartition createPartition(DLearnPartitionStorage storage) { + return new DLearnLabeledDatasetPartition<>(storage); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java new file mode 100644 index 0000000000000..efcc87556d542 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java @@ -0,0 +1,50 @@ +/* + * 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.ml.dlearn.dataset; + +import org.apache.ignite.ml.dlearn.DLearnContext; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; +import org.apache.ignite.ml.math.functions.IgniteBiConsumer; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; + +/** */ +public class AbstractDLearnContextWrapper

implements DLearnContext

{ + /** */ + protected final DLearnContext

delegate; + + /** */ + public AbstractDLearnContextWrapper(DLearnContext

delegate) { + this.delegate = delegate; + } + + /** */ + @Override public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer) { + return delegate.compute(mapper, reducer); + } + + /** */ + @Override public void compute(IgniteBiConsumer mapper) { + delegate.compute(mapper); + } + + /** */ + @Override public > C transform(DLearnContextTransformer transformer) { + return delegate.transform(transformer); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DatasetMathUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnDataset.java similarity index 50% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DatasetMathUtils.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnDataset.java index 7406ae2c92510..cb385b29b3fd9 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DatasetMathUtils.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnDataset.java @@ -19,16 +19,21 @@ import com.github.fommil.netlib.BLAS; import org.apache.ignite.ml.dlearn.DLearnContext; -import org.apache.ignite.ml.dlearn.part.DatasetDLeanPartition; +import org.apache.ignite.ml.dlearn.dataset.part.DLeanDatasetPartition; /** */ -public class DatasetMathUtils { +public class DLearnDataset

extends AbstractDLearnContextWrapper

{ /** */ private static final BLAS blas = BLAS.getInstance(); /** */ - public static double[] mean(DLearnContext learningCtx, int[] cols) { - ValueWithCount res = learningCtx.compute((part, partIdx) -> { + public DLearnDataset(DLearnContext

delegate) { + super(delegate); + } + + /** */ + public double[] mean(int[] cols) { + ValueWithCount res = delegate.compute((part, partIdx) -> { double[] features = part.getFeatures(); int m = part.getRows(); double[] y = new double[cols.length]; @@ -42,14 +47,14 @@ public static double[] mean(DLearnContext learn } /** */ - public static double mean(DLearnContext learningCtx, int col) { - return mean(learningCtx, new int[]{col})[0]; + public double mean(int col) { + return mean(new int[]{col})[0]; } /** */ - public static double[] std(DLearnContext learningCtx, int[] cols) { - double[] mean = mean(learningCtx, cols); - ValueWithCount res = learningCtx.compute(part -> { + public double[] std(int[] cols) { + double[] mean = mean(cols); + ValueWithCount res = delegate.compute(part -> { double[] features = part.getFeatures(); int m = part.getRows(); double[] y = new double[cols.length]; @@ -64,14 +69,62 @@ public static double[] std(DLearnContext learni return res.val; } - public static double std(DLearnContext learningCtx, int col) { - return std(learningCtx, new int[]{col})[0]; + /** */ + public double std(int col) { + return std(new int[]{col})[0]; + } + + /** */ + public double[][] cov(int[] cols) { + double[] mean = mean(cols); + ValueWithCount res = delegate.compute(part -> { + double[] features = part.getFeatures(); + int m = part.getRows(); + double[][] y = new double[cols.length][cols.length]; + for (int i = 0; i < cols.length; i++) + for (int j = 0; j < cols.length; j++) { + int firstCol = cols[i]; + int secondCol = cols[j]; + for (int k = 0; k < m; k++) { + double firstVal = features[m * firstCol + k]; + double secondVal = features[m * secondCol + k]; + y[i][j] += ((firstVal - mean[firstCol]) * (secondVal - mean[secondCol])); + } + } + return new ValueWithCount<>(y, m); + }, (a, b) -> a == null ? b : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); + return scale(res.val, 1.0 / res.cnt); + } + + /** */ + public double[][] corr(int[] cols) { + double[][] cov = cov(cols); + double[] std = std(cols); + for (int i = 0; i < cov.length; i++) + for (int j = 0; j < cov[0].length; j++) + cov[i][j] /= (std[i]*std[j]); + return cov; } /** */ private static double[] sum(double[] a, double[] b) { - blas.daxpy(a.length, 1.0, a, 1, b, 1); - return b; + for (int i = 0; i < a.length; i++) + a[i] += b[i]; + return a; + } + + private static double[][] sum(double[][] a, double[][] b) { + for (int i = 0; i < a.length; i++) + for (int j = 0; j < a[i].length; j++) + a[i][j] += b[i][j]; + return a; + } + + private static double[][] scale(double[][] a, double alpha) { + for (int i = 0; i < a.length; i++) + for (int j = 0; j < a[i].length; j++) + a[i][j] *= alpha; + return a; } /** */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnLabeledDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnLabeledDataset.java new file mode 100644 index 0000000000000..3a84519639035 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnLabeledDataset.java @@ -0,0 +1,29 @@ +/* + * 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.ml.dlearn.dataset; + +import org.apache.ignite.ml.dlearn.DLearnContext; +import org.apache.ignite.ml.dlearn.dataset.part.DLearnLabeledDatasetPartition; + +public class DLearnLabeledDataset extends DLearnDataset> { + /** */ + public DLearnLabeledDataset( + DLearnContext> delegate) { + super(delegate); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/DatasetDLeanPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLeanDatasetPartition.java similarity index 92% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/DatasetDLeanPartition.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLeanDatasetPartition.java index 50f00fe476210..440f237cd7356 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/DatasetDLeanPartition.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLeanDatasetPartition.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlearn.part; +package org.apache.ignite.ml.dlearn.dataset.part; import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; /** * Interface which provides simple dataset API which allows to get or set an underlying feature matrix in flat format. */ -public class DatasetDLeanPartition { +public class DLeanDatasetPartition { /** */ private static final String FEATURES_KEY = "features"; @@ -33,7 +33,7 @@ public class DatasetDLeanPartition { private final DLearnPartitionStorage storage; /** */ - public DatasetDLeanPartition(DLearnPartitionStorage storage) { + public DLeanDatasetPartition(DLearnPartitionStorage storage) { this.storage = storage; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/LabeledDatasetDLearnPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLearnLabeledDatasetPartition.java similarity index 88% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/LabeledDatasetDLearnPartition.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLearnLabeledDatasetPartition.java index fdaa3e2fbd036..f58fa1f80fe79 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/LabeledDatasetDLearnPartition.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLearnLabeledDatasetPartition.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlearn.part; +package org.apache.ignite.ml.dlearn.dataset.part; import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; @@ -23,7 +23,7 @@ * Interface which provides simple dataset API which allows to get or set an underlying feature matrix in flat format * and vector of labels. */ -public class LabeledDatasetDLearnPartition extends DatasetDLeanPartition { +public class DLearnLabeledDatasetPartition extends DLeanDatasetPartition { /** */ private static final String LABELS_KEY = "labels"; @@ -31,7 +31,7 @@ public class LabeledDatasetDLearnPartition extends DatasetDLeanPartition { private final DLearnPartitionStorage storage; /** */ - public LabeledDatasetDLearnPartition(DLearnPartitionStorage storage) { + public DLearnLabeledDatasetPartition(DLearnPartitionStorage storage) { super(storage); this.storage = storage; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/package-info.java similarity index 94% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/package-info.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/package-info.java index 8935df016e1ca..f47f3c903cdf9 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/part/package-info.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/package-info.java @@ -19,4 +19,4 @@ * * Root package for different d-learn partitions. */ -package org.apache.ignite.ml.dlearn.part; \ No newline at end of file +package org.apache.ignite.ml.dlearn.dataset.part; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextTransformer.java deleted file mode 100644 index ba208e28d7856..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextTransformer.java +++ /dev/null @@ -1,29 +0,0 @@ -package org.apache.ignite.ml.dlearn.utils; - -import org.apache.ignite.ml.dlearn.DLearnPartitionFactory; -import org.apache.ignite.ml.math.functions.IgniteBiConsumer; - -/** */ -public class DLearnContextTransformer { - /** */ - private final IgniteBiConsumer transformer; - - /** */ - private final DLearnPartitionFactory partFactory; - - /** */ - public DLearnContextTransformer(IgniteBiConsumer transformer, DLearnPartitionFactory partFactory) { - this.transformer = transformer; - this.partFactory = partFactory; - } - - /** */ - public IgniteBiConsumer getTransformer() { - return transformer; - } - - /** */ - public DLearnPartitionFactory getPartFactory() { - return partFactory; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java index 483b0cf3fc0d2..e3496b6b6084e 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java @@ -19,15 +19,15 @@ import org.apache.ignite.ml.Trainer; import org.apache.ignite.ml.dlearn.DLearnContext; -import org.apache.ignite.ml.dlearn.part.LabeledDatasetDLearnPartition; +import org.apache.ignite.ml.dlearn.dataset.part.DLearnLabeledDatasetPartition; /** */ -public class LinearRegressionLSQRTrainer implements Trainer> { +public class LinearRegressionLSQRTrainer implements Trainer> { /** */ private static final String A_NAME = "a"; /** */ - @Override public LinearRegressionModel train(DLearnContext learningCtx) { + @Override public LinearRegressionModel train(DLearnContext learningCtx) { preProcessContext(learningCtx); // DistributedLSQR lsqr = new DistributedLSQR<>( @@ -51,7 +51,7 @@ public class LinearRegressionLSQRTrainer implements Trainer learningCtx) { + private void preProcessContext(DLearnContext learningCtx) { learningCtx.compute(part -> { double[] features = part.getFeatures(); @@ -65,7 +65,7 @@ private void preProcessContext(DLearnContext learningCtx) { + private void postProcessContext(DLearnContext learningCtx) { // learningCtx.compute(part -> part.remove(A_NAME)); } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java b/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java index bd7c41d12e73b..c0e33f9764c9d 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java @@ -27,9 +27,8 @@ import org.apache.ignite.ml.dlearn.DLearnContext; import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContextFactory; import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; -import org.apache.ignite.ml.dlearn.dataset.DatasetMathUtils; -import org.apache.ignite.ml.dlearn.part.LabeledDatasetDLearnPartition; import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformers; +import org.apache.ignite.ml.dlearn.dataset.DLearnLabeledDataset; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; /** */ @@ -75,7 +74,8 @@ public void testTrainOnBostonDataset() { // In this case for every partition in upstream cache will be created labeled dataset partition and this new // partition will be filled with help of specified feature and label extractors. - DLearnContext> datasetLearningCtx = cacheLearningCtx + + DLearnLabeledDataset dataset = cacheLearningCtx .transform( DLearnContextTransformers.cacheToLabeledDataset( (k, v) -> Arrays.copyOfRange(v, 1, v.length), // specify feature extractor @@ -84,11 +84,11 @@ public void testTrainOnBostonDataset() { ); // Calculation of mean value. This calculation will be performed in map-reduce manner. - double[] mean = DatasetMathUtils.mean(datasetLearningCtx, new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); + double[] mean = dataset.mean(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); System.err.println("Mean values : " + Arrays.toString(mean)); // Calculation of standard deviation. This calculation will be performed in map-reduce manner. - double[] std = DatasetMathUtils.std(datasetLearningCtx, new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); + double[] std = dataset.std(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); System.err.println("Std values : " + Arrays.toString(std)); } From c03e58ef67debc2a2e42bfd9f829bfe5c5a60a45 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Wed, 17 Jan 2018 13:30:16 +0300 Subject: [PATCH 05/38] IGNITE-7437 Add ability to close learning context. --- .../ignite/ml/dlearn/DLearnContext.java | 9 +- .../ml/dlearn/DLearnContextFactory.java | 2 +- .../ml/dlearn/DLearnPartitionFactory.java | 2 +- .../context/cache/CacheDLearnContext.java | 32 +++- .../cache/CacheDLearnContextFactory.java | 2 +- .../context/cache/CacheDLearnPartition.java | 10 +- .../DLearnPartitionAffinityFunction.java | 29 +++- .../context/local/LocalDLearnContext.java | 30 +++- .../local/LocalDLearnContextFactory.java | 16 +- .../context/local/LocalDLearnPartition.java | 15 +- .../transformer/DLearnContextTransformer.java | 2 +- .../DLearnContextTransformers.java | 5 +- ...ocalDatasetDLearnPartitionTransformer.java | 22 +-- ...eledDatasetDLearnPartitionTransformer.java | 30 ++-- .../dataset/AbstractDLearnContextWrapper.java | 15 +- .../dataset/part/DLeanDatasetPartition.java | 10 +- .../part/DLearnLabeledDatasetPartition.java | 8 + .../java/org/apache/ignite/ml/Playground.java | 13 +- .../context/cache/CacheDLearnContextTest.java | 151 ++++++++++++++++++ .../context/local/LocalDLearnContextTest.java | 125 +++++++++++++++ 20 files changed, 460 insertions(+), 68 deletions(-) create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextTest.java diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java index 5ea9dff111d10..0bf0624ca73a9 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java @@ -30,7 +30,7 @@ * * @param

type of learning context partition */ -public interface DLearnContext

{ +public interface DLearnContext

extends AutoCloseable { /** * Computes a given function on every d-learn partition in current learning context independently and then reduces * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes @@ -87,5 +87,10 @@ default public void compute(IgniteConsumer

mapper) { * @param type of new d-learn partition * @return new learning context */ - public > C transform(DLearnContextTransformer transformer); + public > C transform(DLearnContextTransformer transformer); + + /** + * Removes all data associated with the context. + */ + public void close(); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContextFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContextFactory.java index 6ceec65b4252e..cfb047cf0e6a1 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContextFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContextFactory.java @@ -25,7 +25,7 @@ * @param

type of d-learn partition */ @FunctionalInterface -public interface DLearnContextFactory

extends Serializable { +public interface DLearnContextFactory

extends Serializable { /** * Creates new instance of learning context. * diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionFactory.java index f99226fb5d374..9e922345b6719 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionFactory.java @@ -27,7 +27,7 @@ * @param

type of d-learn partition */ @FunctionalInterface -public interface DLearnPartitionFactory

extends Serializable { +public interface DLearnPartitionFactory

extends Serializable { /** * Creates a new d-learn partition (type-safe wrapper on top of a partition storage). * diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java index d13f385617eab..fd9d6907dff70 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java @@ -38,7 +38,7 @@ * * @param

type of learning context partition */ -public class CacheDLearnContext

implements DLearnContext

{ +public class CacheDLearnContext

implements DLearnContext

{ /** */ private final Ignite ignite; @@ -59,7 +59,7 @@ public CacheDLearnContext(Ignite ignite, String learningCtxCacheName, DLearnPart this.learningCtxId = learningCtxId; } - /** */ + /** {@inheritDoc} */ public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer) { ClusterGroup clusterGrp = ignite.cluster().forDataNodes(learningCtxCacheName); @@ -83,7 +83,7 @@ public R compute(IgniteBiFunction mapper, IgniteBinaryOperato return reduce(results, reducer); } - /** */ + /** {@inheritDoc} */ @Override public void compute(IgniteBiConsumer mapper) { ClusterGroup clusterGrp = ignite.cluster().forDataNodes(learningCtxCacheName); @@ -102,8 +102,8 @@ public R compute(IgniteBiFunction mapper, IgniteBinaryOperato }); } - /** */ - @Override public > C transform(DLearnContextTransformer transformer) { + /** {@inheritDoc} */ + @Override public > C transform(DLearnContextTransformer transformer) { UUID newLearningCtxId = UUID.randomUUID(); compute((part, partIdx) -> { @@ -125,4 +125,24 @@ private R reduce(Collection results, IgniteBinaryOperator reducer) { res = reducer.apply(res, partRes); return res; } -} \ No newline at end of file + + /** {@inheritDoc} */ + @Override public void close() { + compute(this::closePartition); + } + + /** */ + private void closePartition(P part) { + try { + part.close(); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** */ + public String getLearningCtxCacheName() { + return learningCtxCacheName; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java index 55c96f0f4ac55..be49fab6d3f30 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java @@ -103,6 +103,6 @@ private DLearnPartitionAffinityFunction createLearningContextCacheAffinityFuncti break; } - return new DLearnPartitionAffinityFunction(initAssignment, topVer); + return new DLearnPartitionAffinityFunction(initAssignment, topVer, 0); } } \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java index b74e24ed88c82..a19ea59814ea4 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java @@ -20,7 +20,7 @@ import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; /** */ -public class CacheDLearnPartition { +public class CacheDLearnPartition implements AutoCloseable { /** */ private static final String UPSTREAM_CACHE_NAME_KEY = "upstream_cache_name"; @@ -54,4 +54,12 @@ public void setPart(int part) { public int getPart() { return storage.get(PART_KEY); } + + /** + * Removes all data associated with the partition. + */ + @Override public void close() { + storage.remove(UPSTREAM_CACHE_NAME_KEY); + storage.remove(PART_KEY); + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunction.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunction.java index dac086f3ed475..05ea4a1eb5adb 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunction.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunction.java @@ -30,28 +30,41 @@ import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; /** - * Affinity function used to identify partition number and node to place learning context partition. This function is - * initialized with {@link #initAssignment} parameter which contains information about upstream cache distribution across - * cluster. It allows it to place learning context partitions on the same nodes as partitions of the upstream cache. + * This affinity function is used to identify a partition by key and node to place the partition. This function is + * initialized with {@link #initAssignment} parameter which contains information about upstream cache distribution + * across the cluster. This information allows function to place context partitions on the same nodes as partitions + * of the upstream cache. Be aware that this affinity functions supports only {@link DLearnContextPartitionKey} keys. */ public class DLearnPartitionAffinityFunction implements AffinityFunction { /** */ private static final long serialVersionUID = 7735390384525189270L; /** - * Initial distribution of the partitions (copy of upstream cache partitions distribution) + * Initial distribution of the partitions (copy of upstream cache partitions distribution). */ private final List initAssignment; /** - * Version of the topology used to make an {@link #initAssignment} + * Version of the topology used to collect the {@link #initAssignment}. */ private final long initTopVer; - /** */ - public DLearnPartitionAffinityFunction(List initAssignment, long initTopVer) { + /** + * Number of partition backups. + */ + private final int backups; + + /** + * Creates new instance of d-learn partition affinity function initialized with initial distribution. + * + * @param initAssignment initial distribution of the partitions (copy of upstream cache partitions distribution) + * @param initTopVer version of the topology used to collect the {@link #initAssignment} + * @param backups number of partition backups + */ + public DLearnPartitionAffinityFunction(List initAssignment, long initTopVer, int backups) { this.initAssignment = initAssignment; this.initTopVer = initTopVer; + this.backups = backups; } /** {@inheritDoc} */ @@ -66,7 +79,7 @@ public DLearnPartitionAffinityFunction(List initAssignment, long initTopVe /** {@inheritDoc} */ @Override public int partition(Object key) { - BinaryObject bo = (BinaryObject)key; + BinaryObject bo = (BinaryObject) key; DLearnContextPartitionKey datasetPartKey = bo.deserialize(); return datasetPartKey.getPart(); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java index a40ba2f50774a..51b5902c4447b 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java @@ -33,7 +33,7 @@ * * @param

type of learning context partition */ -public class LocalDLearnContext

implements DLearnContext

{ +public class LocalDLearnContext

implements DLearnContext

{ /** */ private final Map learningCtxMap; @@ -55,7 +55,7 @@ public LocalDLearnContext(Map learningCtxMap, this.partitions = partitions; } - /** */ + /** {@inheritDoc} */ @Override public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer) { R res = null; for (int partIdx = 0; partIdx < partitions; partIdx++) { @@ -67,7 +67,7 @@ public LocalDLearnContext(Map learningCtxMap, return res; } - /** */ + /** {@inheritDoc} */ @Override public void compute(IgniteBiConsumer mapper) { for (int partIdx = 0; partIdx < partitions; partIdx++) { DLearnPartitionStorage storage = new LocalDLearnPartitionStorage(learningCtxMap, learningCtxId, partIdx); @@ -76,8 +76,8 @@ public LocalDLearnContext(Map learningCtxMap, } } - /** */ - @Override public > C transform(DLearnContextTransformer transformer) { + /** {@inheritDoc} */ + @Override public > C transform(DLearnContextTransformer transformer) { UUID newLearningCtxId = UUID.randomUUID(); compute((part, partIdx) -> { @@ -90,4 +90,24 @@ public LocalDLearnContext(Map learningCtxMap, return transformer.wrapContext(newCtx); } + + /** {@inheritDoc} */ + @Override public void close() { + compute(this::closePartition); + } + + /** */ + private void closePartition(P part) { + try { + part.close(); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** */ + public Map getLearningCtxMap() { + return learningCtxMap; + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java index 5945b9c788c5f..e935b4726c87b 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java @@ -31,37 +31,39 @@ * * @param type of upstream values */ -public class LocalDLearnContextFactory implements DLearnContextFactory> { +public class LocalDLearnContextFactory implements DLearnContextFactory> { /** */ private static final long serialVersionUID = -7614441997952907675L; /** */ - private final List data; + private final Map data; /** */ private final int partitions; /** */ - public LocalDLearnContextFactory(List data, int partitions) { + public LocalDLearnContextFactory(Map data, int partitions) { this.data = data; this.partitions = partitions; } /** {@inheritDoc} */ - @Override public LocalDLearnContext> createContext() { + @Override public LocalDLearnContext> createContext() { Map learningCtxMap = new HashMap<>(); + UUID learningCtxId = UUID.randomUUID(); int partSize = data.size() / partitions; // loads data into learning context partitions + List keys = new ArrayList<>(data.keySet()); for (int partIdx = 0; partIdx < partitions; partIdx++) { - List partData = new ArrayList<>(); + Map partData = new HashMap<>(); for (int j = partIdx * partSize; j < (partIdx + 1) * partSize && j < data.size(); j++) - partData.add(data.get(j)); + partData.put(keys.get(j), data.get(keys.get(j))); DLearnPartitionStorage storage = new LocalDLearnPartitionStorage(learningCtxMap, learningCtxId, partIdx); - LocalDLearnPartition part = new LocalDLearnPartition<>(storage); + LocalDLearnPartition part = new LocalDLearnPartition<>(storage); part.setPartData(partData); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java index 664929299d76d..42108f3b467f1 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java @@ -17,13 +17,13 @@ package org.apache.ignite.ml.dlearn.context.local; -import java.util.List; +import java.util.Map; import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; /** * Learning context partition which uses local on-heap hash map to keep data. */ -public class LocalDLearnPartition { +public class LocalDLearnPartition implements AutoCloseable { /** */ private static final String PART_DATA_KEY = "part_data"; @@ -36,12 +36,19 @@ public LocalDLearnPartition(DLearnPartitionStorage storage) { } /** */ - public List getPartData() { + public Map getPartData() { return storage.get(PART_DATA_KEY); } /** */ - public void setPartData(List partData) { + public void setPartData(Map partData) { storage.put(PART_DATA_KEY, partData); } + + /** + * Removes all data associated with the partition. + */ + @Override public void close() { + storage.remove(PART_DATA_KEY); + } } \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformer.java index 6f7263f88a8c2..bf2c6229d0a28 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformer.java @@ -4,7 +4,7 @@ import org.apache.ignite.ml.dlearn.DLearnPartitionFactory; /** */ -public interface DLearnContextTransformer> extends DLearnPartitionFactory { +public interface DLearnContextTransformer

> extends DLearnPartitionFactory { /** */ public void transform(P oldPart, T newPart); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java index ed5463c386bc0..996338cc7f682 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java @@ -5,7 +5,6 @@ import org.apache.ignite.ml.dlearn.context.transformer.local.LocalDatasetDLearnPartitionTransformer; import org.apache.ignite.ml.dlearn.context.transformer.local.LocalLabeledDatasetDLearnPartitionTransformer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteFunction; /** */ public class DLearnContextTransformers { @@ -20,12 +19,12 @@ public static CacheLabeledDatasetDLearnPartitionTransformer c } /** */ - public static LocalDatasetDLearnPartitionTransformer localToDataset(IgniteFunction featureExtractor) { + public static LocalDatasetDLearnPartitionTransformer localToDataset(IgniteBiFunction featureExtractor) { return new LocalDatasetDLearnPartitionTransformer<>(featureExtractor); } /** */ - public static LocalLabeledDatasetDLearnPartitionTransformer localToLabeledDataset(IgniteFunction featureExtractor, IgniteFunction lbExtractor) { + public static LocalLabeledDatasetDLearnPartitionTransformer localToLabeledDataset(IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { return new LocalLabeledDatasetDLearnPartitionTransformer<>(featureExtractor, lbExtractor); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java index ff16fa980ed46..0684d25865ced 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java @@ -17,37 +17,41 @@ package org.apache.ignite.ml.dlearn.context.transformer.local; +import java.util.ArrayList; import java.util.List; +import java.util.Map; import org.apache.ignite.ml.dlearn.DLearnContext; import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; import org.apache.ignite.ml.dlearn.dataset.part.DLeanDatasetPartition; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; -import org.apache.ignite.ml.math.functions.IgniteFunction; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; /** */ -public class LocalDatasetDLearnPartitionTransformer - implements DLearnContextTransformer, DLeanDatasetPartition, DLearnDataset> { +public class LocalDatasetDLearnPartitionTransformer + implements DLearnContextTransformer, DLeanDatasetPartition, DLearnDataset> { /** */ private static final long serialVersionUID = -7567051002880704559L; /** */ - private final IgniteFunction featureExtractor; + private final IgniteBiFunction featureExtractor; /** */ - public LocalDatasetDLearnPartitionTransformer(IgniteFunction featureExtractor) { + public LocalDatasetDLearnPartitionTransformer(IgniteBiFunction featureExtractor) { this.featureExtractor = featureExtractor; } /** */ - @Override public void transform(LocalDLearnPartition oldPart, DLeanDatasetPartition newPart) { - List partData = oldPart.getPartData(); + @Override public void transform(LocalDLearnPartition oldPart, DLeanDatasetPartition newPart) { + Map partData = oldPart.getPartData(); if (partData != null && !partData.isEmpty()) { double[] features = null; int m = partData.size(), n = 0; + List keys = new ArrayList<>(partData.keySet()); for (int i = 0; i < partData.size(); i++) { - double[] rowFeatures = featureExtractor.apply(partData.get(i)); + K key = keys.get(i); + double[] rowFeatures = featureExtractor.apply(key, partData.get(key)); if (i == 0) { n = rowFeatures.length; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java index 054c4dcd82029..a6cd558a0e3a8 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java @@ -17,42 +17,46 @@ package org.apache.ignite.ml.dlearn.context.transformer.local; +import java.util.ArrayList; import java.util.List; +import java.util.Map; import org.apache.ignite.ml.dlearn.DLearnContext; import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; import org.apache.ignite.ml.dlearn.dataset.DLearnLabeledDataset; import org.apache.ignite.ml.dlearn.dataset.part.DLearnLabeledDatasetPartition; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; -import org.apache.ignite.ml.math.functions.IgniteFunction; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; /** */ -public class LocalLabeledDatasetDLearnPartitionTransformer - implements DLearnContextTransformer,DLearnLabeledDatasetPartition, DLearnLabeledDataset> { +public class LocalLabeledDatasetDLearnPartitionTransformer + implements DLearnContextTransformer,DLearnLabeledDatasetPartition, DLearnLabeledDataset> { /** */ private static final long serialVersionUID = -8438445094768312331L; /** */ - private final IgniteFunction featureExtractor; + private final IgniteBiFunction featureExtractor; /** */ - private final IgniteFunction lbExtractor; + private final IgniteBiFunction lbExtractor; /** */ - public LocalLabeledDatasetDLearnPartitionTransformer(IgniteFunction featureExtractor, IgniteFunction lbExtractor) { + public LocalLabeledDatasetDLearnPartitionTransformer(IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { this.featureExtractor = featureExtractor; this.lbExtractor = lbExtractor; } /** */ @SuppressWarnings("unchecked") - @Override public void transform(LocalDLearnPartition oldPart, DLearnLabeledDatasetPartition newPart) { - List partData = oldPart.getPartData(); + @Override public void transform(LocalDLearnPartition oldPart, DLearnLabeledDatasetPartition newPart) { + Map partData = oldPart.getPartData(); if (partData != null && !partData.isEmpty()) { double[] features = null; int m = partData.size(), n = 0; + List keys = new ArrayList<>(partData.keySet()); for (int i = 0; i < partData.size(); i++) { - double[] rowFeatures = featureExtractor.apply(partData.get(i)); + K key = keys.get(i); + double[] rowFeatures = featureExtractor.apply(key, partData.get(i)); if (i == 0) { n = rowFeatures.length; @@ -69,8 +73,10 @@ public LocalLabeledDatasetDLearnPartitionTransformer(IgniteFunction newPart.setRows(m); L[] labels = (L[]) new Object[partData.size()]; - for (int i = 0; i < partData.size(); i++) - labels[i] = lbExtractor.apply(partData.get(i)); + for (int i = 0; i < partData.size(); i++) { + K key = keys.get(i); + labels[i] = lbExtractor.apply(key, partData.get(key)); + } newPart.setLabels(labels); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java index efcc87556d542..73cc8a90a5ccb 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java @@ -24,7 +24,7 @@ import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; /** */ -public class AbstractDLearnContextWrapper

implements DLearnContext

{ +public class AbstractDLearnContextWrapper

implements DLearnContext

{ /** */ protected final DLearnContext

delegate; @@ -33,18 +33,23 @@ public AbstractDLearnContextWrapper(DLearnContext

delegate) { this.delegate = delegate; } - /** */ + /** {@inheritDoc} */ @Override public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer) { return delegate.compute(mapper, reducer); } - /** */ + /** {@inheritDoc} */ @Override public void compute(IgniteBiConsumer mapper) { delegate.compute(mapper); } - /** */ - @Override public > C transform(DLearnContextTransformer transformer) { + /** {@inheritDoc} */ + @Override public > C transform(DLearnContextTransformer transformer) { return delegate.transform(transformer); } + + /** {@inheritDoc} */ + @Override public void close() { + delegate.close(); + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLeanDatasetPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLeanDatasetPartition.java index 440f237cd7356..e0cc753af6c2e 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLeanDatasetPartition.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLeanDatasetPartition.java @@ -22,7 +22,7 @@ /** * Interface which provides simple dataset API which allows to get or set an underlying feature matrix in flat format. */ -public class DLeanDatasetPartition { +public class DLeanDatasetPartition implements AutoCloseable { /** */ private static final String FEATURES_KEY = "features"; @@ -72,4 +72,12 @@ public void setRows(int rows) { public int getRows() { return storage.get(ROWS_KEY); } + + /** + * Removes all data associated with the partition. + */ + @Override public void close() { + storage.remove(FEATURES_KEY); + storage.remove(ROWS_KEY); + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLearnLabeledDatasetPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLearnLabeledDatasetPartition.java index f58fa1f80fe79..86b930e3a7d30 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLearnLabeledDatasetPartition.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLearnLabeledDatasetPartition.java @@ -53,4 +53,12 @@ public void setLabels(L[] labels) { public L[] getLabels() { return storage.get(LABELS_KEY); } + + /** + * Removes all data associated with the partition. + */ + @Override public void close() { + super.close(); + storage.remove(LABELS_KEY); + } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java b/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java index c0e33f9764c9d..a37714e3a573e 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java @@ -63,7 +63,6 @@ public class Playground extends GridCommonAbstractTest { /** */ public void testTrainOnBostonDataset() { IgniteCache bostonDataset = loadDataset(); - // Initialization of d-learn context, after this step context cache will be created with partitions placed on // the same nodes as the upstream Ignite Cache (in this case bostonDataset). DLearnContext> cacheLearningCtx = @@ -86,10 +85,22 @@ public void testTrainOnBostonDataset() { // Calculation of mean value. This calculation will be performed in map-reduce manner. double[] mean = dataset.mean(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); System.err.println("Mean values : " + Arrays.toString(mean)); + TestUtils.assertEquals(new double[]{3.59376071e+00, 1.13636364e+01, 1.11367787e+01, 6.91699605e-02, + 5.54695059e-01, 6.28463439e+00, 6.85749012e+01, 3.79504269e+00, + 9.54940711e+00, 4.08237154e+02}, mean, 1e-6); // Calculation of standard deviation. This calculation will be performed in map-reduce manner. double[] std = dataset.std(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); System.err.println("Std values : " + Arrays.toString(std)); + TestUtils.assertEquals(new double[]{8.58828355e+00, 2.32993957e+01, 6.85357058e+00, 2.53742935e-01, + 1.15763115e-01, 7.01922514e-01, 2.81210326e+01, 2.10362836e+00, + 8.69865112e+00, 1.68370495e+02}, std, 1e-6); + + // Calculation of covariance matrix. This calculation will be performed in map-reduce manner. + double[][] cov = dataset.cov(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); + System.err.println("Covariance matrix : "); + for (double[] row : cov) + System.err.println(Arrays.toString(row)); } /** */ diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java new file mode 100644 index 0000000000000..c5e0c88f2ee20 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java @@ -0,0 +1,151 @@ +/* + * 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.ml.dlearn.context.cache; + +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformers; +import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; +import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests for {@link CacheDLearnContext}. + */ +public class CacheDLearnContextTest extends GridCommonAbstractTest { + /** Number of nodes in grid */ + private static final int NODE_COUNT = 4; + + /** */ + private Ignite ignite; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + for (int i = 1; i <= NODE_COUNT; i++) + startGrid(i); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() { + stopAllGrids(); + } + + /** + * {@inheritDoc} + */ + @Override protected void beforeTest() throws Exception { + /* Grid instance. */ + ignite = grid(NODE_COUNT); + ignite.configuration().setPeerClassLoadingEnabled(true); + IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); + } + + /** */ + public void testClose() { + IgniteCache data = generateTestData(); + + CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); + CacheDLearnContext> ctx = ctxFactory.createContext(); + IgniteCache learningCtxCache = ignite.cache(ctx.getLearningCtxCacheName()); + + // context cache contains 2 partitions, each partition contains cache name and partition number + assertEquals(4, learningCtxCache.size()); + + ctx.close(); + + // all data were removed from context cache + assertEquals(0, learningCtxCache.size()); + } + + /** */ + public void testCloseDerivativeContext() { + IgniteCache data = generateTestData(); + + CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); + CacheDLearnContext> ctx = ctxFactory.createContext(); + IgniteCache learningCtxCache = ignite.cache(ctx.getLearningCtxCacheName()); + + // context cache contains 2 partitions, each partition contains cache name and partition number + assertEquals(4, learningCtxCache.size()); + + DLearnDataset dataset = ctx.transform(DLearnContextTransformers.cacheToDataset((k, v) -> new double[0])); + + // features and rows were added into both partitions + assertEquals(8, learningCtxCache.size()); + + dataset.close(); + + // features and rows were removed + assertEquals(4, learningCtxCache.size()); + + ctx.close(); + + // all data were removed from context cache + assertEquals(0, learningCtxCache.size()); + } + + /** */ + public void testCloseBaseContext() { + IgniteCache data = generateTestData(); + + CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); + CacheDLearnContext> ctx = ctxFactory.createContext(); + IgniteCache learningCtxCache = ignite.cache(ctx.getLearningCtxCacheName()); + + // context cache contains 2 partitions, each partition contains cache name and partition number + assertEquals(4, learningCtxCache.size()); + + DLearnDataset dataset = ctx.transform(DLearnContextTransformers.cacheToDataset((k, v) -> new double[0])); + + // features and rows were added into both partitions + assertEquals(8, learningCtxCache.size()); + + ctx.close(); + + // 2 partitions with initial data were removed + assertEquals(4, learningCtxCache.size()); + + dataset.close(); + + // all data were removed from context cache + assertEquals(0, learningCtxCache.size()); + } + + /** + * Generates Ignite Cache with data for tests. + * + * @return Ignite Cache with data for tests + */ + private IgniteCache generateTestData() { + CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); + cacheConfiguration.setName(UUID.randomUUID().toString()); + cacheConfiguration.setAffinity(new RendezvousAffinityFunction(true, 2)); + + IgniteCache cache = ignite.createCache(cacheConfiguration); + cache.put(1, "TEST1"); + cache.put(2, "TEST2"); + cache.put(3, "TEST3"); + cache.put(4, "TEST4"); + + return cache; + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextTest.java new file mode 100644 index 0000000000000..dd4bc3b29bd90 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextTest.java @@ -0,0 +1,125 @@ +/* + * 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.ml.dlearn.context.local; + +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformers; +import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; +import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link LocalDLearnContext}. + */ +public class LocalDLearnContextTest { + /** */ + @Test + public void testClose() { + Map data = generateTestData(); + + LocalDLearnContextFactory ctxFactory = new LocalDLearnContextFactory<>(data, 2); + + LocalDLearnContext> ctx = ctxFactory.createContext(); + Map learningCtxMap = ctx.getLearningCtxMap(); + + // context cache contains 2 partitions with initial data + assertEquals(2, learningCtxMap.size()); + + ctx.close(); + + // all data were removed from context cache + assertEquals(0, learningCtxMap.size()); + } + + /** */ + @Test + public void testCloseDerivativeContext() { + Map data = generateTestData(); + + LocalDLearnContextFactory ctxFactory = new LocalDLearnContextFactory<>(data, 2); + + LocalDLearnContext> ctx = ctxFactory.createContext(); + Map learningCtxMap = ctx.getLearningCtxMap(); + + // context cache contains 2 partitions with initial data + assertEquals(2, learningCtxMap.size()); + + DLearnDataset dataset = ctx.transform(DLearnContextTransformers.localToDataset((k, v) -> new double[0])); + + // features and rows were added into both partitions + assertEquals(6, learningCtxMap.size()); + + dataset.close(); + + // features and rows were removed + assertEquals(2, learningCtxMap.size()); + + ctx.close(); + + // all data were removed from context cache + assertEquals(0, learningCtxMap.size()); + } + + /** */ + @Test + public void testCloseBaseContext() { + Map data = generateTestData(); + + LocalDLearnContextFactory ctxFactory = new LocalDLearnContextFactory<>(data, 2); + + LocalDLearnContext> ctx = ctxFactory.createContext(); + Map learningCtxMap = ctx.getLearningCtxMap(); + + // context cache contains 2 partitions with initial data + assertEquals(2, learningCtxMap.size()); + + DLearnDataset dataset = ctx.transform(DLearnContextTransformers.localToDataset((k, v) -> new double[0])); + + // features and rows were added into both partitions + assertEquals(6, learningCtxMap.size()); + + ctx.close(); + + // 2 partitions with initial data were removed + assertEquals(4, learningCtxMap.size()); + + dataset.close(); + + // all data were removed from context cache + assertEquals(0, learningCtxMap.size()); + } + + /** + * Generates list with data for test. + * + * @return list with data for test. + */ + private Map generateTestData() { + Map data = new HashMap<>(); + + data.put(1, "TEST1"); + data.put(2, "TEST2"); + data.put(3, "TEST3"); + data.put(4, "TEST4"); + + return data; + } +} From 8cdc9c516f8a248f177e4a5fe9a49b027aba96e3 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Wed, 17 Jan 2018 14:28:43 +0300 Subject: [PATCH 06/38] IGNITE-7437 Reformat code and javadoc. --- .../ignite/ml/dlearn/DLearnContext.java | 14 ++-- .../ml/dlearn/DLearnContextFactory.java | 6 +- .../ml/dlearn/DLearnPartitionFactory.java | 6 +- .../ml/dlearn/DLearnPartitionStorage.java | 10 +-- .../context/cache/CacheDLearnContext.java | 12 ++-- .../context/cache/CacheDLearnPartition.java | 8 ++- .../cache/CacheDLearnPartitionStorage.java | 13 +++- .../context/local/LocalDLearnContext.java | 9 +-- .../context/local/LocalDLearnPartition.java | 8 ++- .../local/LocalDLearnPartitionStorage.java | 16 ++++- .../ml/dlearn/context/package-info.java | 2 +- .../transformer/DLearnContextTransformer.java | 29 +++++++-- .../DLearnContextTransformers.java | 65 ++++++++++++++++--- ...acheDatasetDLearnPartitionTransformer.java | 33 ++++++++-- ...eledDatasetDLearnPartitionTransformer.java | 36 +++++++--- .../transformer/cache/package-info.java | 22 +++++++ ...ocalDatasetDLearnPartitionTransformer.java | 18 +++-- ...eledDatasetDLearnPartitionTransformer.java | 24 +++++-- .../transformer/local/package-info.java | 22 +++++++ .../context/transformer/package-info.java | 2 +- .../dataset/AbstractDLearnContextWrapper.java | 20 ++++-- .../ml/dlearn/dataset/DLearnDataset.java | 8 ++- .../dlearn/dataset/DLearnLabeledDataset.java | 7 +- .../ml/dlearn/dataset/package-info.java | 22 +++++++ .../utils/DLearnContextPartitionKey.java | 2 +- .../java/org/apache/ignite/ml/Playground.java | 24 +++---- 26 files changed, 344 insertions(+), 94 deletions(-) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/package-info.java diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java index 0bf0624ca73a9..edefbbed3aef0 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java @@ -25,10 +25,11 @@ import org.apache.ignite.ml.math.functions.IgniteFunction; /** - * Learning context is a context maintained during a whole learning process. The context provides an ability to perform - * calculations in map-reduce manner and guarantees maintenance of the partition states between compute calls. + * Learning context is a context maintained during a whole learning or data analysis process. The context provides an + * ability to perform calculations in the map-reduce manner and guarantees maintenance of the partition states between + * {@code compute()} calls. * - * @param

type of learning context partition + * @param

type of a learning context partition */ public interface DLearnContext

extends AutoCloseable { /** @@ -37,7 +38,7 @@ public interface DLearnContext

extends AutoCloseable { * where partitions are placed and do not involve network subsystem where it's possible. * * @param mapper mapper function applied on every partition - * @param reducer reducer of the results + * @param reducer reducer of the results * @param result type * @return final reduced result */ @@ -49,7 +50,7 @@ public interface DLearnContext

extends AutoCloseable { * where partitions are placed and do not involve network subsystem where it's possible. * * @param mapper mapper function applied on every partition - * @param reducer reducer of the results + * @param reducer reducer of the results * @param result type * @return final reduced result */ @@ -87,7 +88,8 @@ default public void compute(IgniteConsumer

mapper) { * @param type of new d-learn partition * @return new learning context */ - public > C transform(DLearnContextTransformer transformer); + public > C transform( + DLearnContextTransformer transformer); /** * Removes all data associated with the context. diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContextFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContextFactory.java index cfb047cf0e6a1..bc0c1503b3129 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContextFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContextFactory.java @@ -20,14 +20,14 @@ import java.io.Serializable; /** - * Factory produces learning context instances. + * Factory produces learning context instances (see {@link DLearnContext}). * - * @param

type of d-learn partition + * @param

type of a d-learn partition */ @FunctionalInterface public interface DLearnContextFactory

extends Serializable { /** - * Creates new instance of learning context. + * Constructs a new instance of learning context. * * @return learning context */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionFactory.java index 9e922345b6719..b30f60ba1e8da 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionFactory.java @@ -20,11 +20,11 @@ import java.io.Serializable; /** - * With assumption that a d-lear partition is the type-safe wrapper on top of a partition storage (distributed or not), - * {@code DLearnPartitionFactory} provides API for instantiation of a new d-learn partition instances based on given + * With assumption that a d-lear partition is a type-safe wrapper on top of a partition storage (distributed or not), + * {@code DLearnPartitionFactory} provides API for instantiation of a new d-learn partition instances based on the given * partition storage. * - * @param

type of d-learn partition + * @param

type of a d-learn partition */ @FunctionalInterface public interface DLearnPartitionFactory

extends Serializable { diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionStorage.java index b375f5140f6f3..a4bf70f0b57e0 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionStorage.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionStorage.java @@ -18,12 +18,12 @@ package org.apache.ignite.ml.dlearn; /** - * D-learn partition storage is a common interface for partition storages both local and distributed. it allows to save, - * retrieve and remove objects identified by keys from the d-learn partition. + * D-learn partition storage is a common interface for partition storages both local and distributed. Partition storage + * allows to save, retrieve and remove objects identified by keys from the d-learn partition. */ public interface DLearnPartitionStorage { /** - * Saves given value in the d-learn partition with given key. + * Saves the given value in the d-learn partition with the given key. * * @param key key * @param val value @@ -32,7 +32,7 @@ public interface DLearnPartitionStorage { public void put(String key, T val); /** - * Retrieves value from the d-learn partition by given key. + * Retrieves value from the d-learn partition by the given key. * * @param key key * @param type of value @@ -41,7 +41,7 @@ public interface DLearnPartitionStorage { public T get(String key); /** - * Removes value from the d-learn partition by given key. + * Removes value from the d-learn partition by the given key. * * @param key key */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java index fd9d6907dff70..f3f07e38d01a8 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java @@ -52,7 +52,8 @@ public class CacheDLearnContext

implements DLearnContex private final UUID learningCtxId; /** */ - public CacheDLearnContext(Ignite ignite, String learningCtxCacheName, DLearnPartitionFactory

partFactory, UUID learningCtxId) { + public CacheDLearnContext(Ignite ignite, String learningCtxCacheName, DLearnPartitionFactory

partFactory, + UUID learningCtxId) { this.ignite = ignite; this.learningCtxCacheName = learningCtxCacheName; this.partFactory = partFactory; @@ -103,17 +104,18 @@ public R compute(IgniteBiFunction mapper, IgniteBinaryOperato } /** {@inheritDoc} */ - @Override public > C transform(DLearnContextTransformer transformer) { - UUID newLearningCtxId = UUID.randomUUID(); + @Override public > C transform( + DLearnContextTransformer transformer) { + UUID newLearnCtxId = UUID.randomUUID(); compute((part, partIdx) -> { IgniteCache learningCtxCache = ignite.cache(learningCtxCacheName); - DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, newLearningCtxId, partIdx); + DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, newLearnCtxId, partIdx); T newPart = transformer.createPartition(storage); transformer.transform(part, newPart); }); - DLearnContext newCtx = new CacheDLearnContext<>(ignite, learningCtxCacheName, transformer, newLearningCtxId); + DLearnContext newCtx = new CacheDLearnContext<>(ignite, learningCtxCacheName, transformer, newLearnCtxId); return transformer.wrapContext(newCtx); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java index a19ea59814ea4..0aace852bd5a5 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java @@ -19,7 +19,13 @@ import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; -/** */ +/** + * D-learn partition which uses Ignite cache to keep data and produces as initial context partition by + * {@link CacheDLearnContextFactory}. + * + * @param type of keys + * @param type of values + */ public class CacheDLearnPartition implements AutoCloseable { /** */ private static final String UPSTREAM_CACHE_NAME_KEY = "upstream_cache_name"; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartitionStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartitionStorage.java index c9b771ab1e8a1..d3c65d841adbe 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartitionStorage.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartitionStorage.java @@ -27,14 +27,22 @@ import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; +/** + * D-learn partition storage based on Ignite cache. + */ public class CacheDLearnPartitionStorage implements DLearnPartitionStorage { - + /** + * Storage. + */ private final IgniteCache learningCtxCache; + /** */ private final UUID learningCtxId; + /** */ private final int part; + /** */ public CacheDLearnPartitionStorage(IgniteCache learningCtxCache, UUID learningCtxId, int part) { this.learningCtxCache = learningCtxCache; @@ -42,15 +50,18 @@ public CacheDLearnPartitionStorage(IgniteCache void put(String key, T val) { learningCtxCache.put(new DLearnContextPartitionKey(part, learningCtxId, key), serialize(val)); } + /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public T get(String key) { return (T) deserialize(learningCtxCache.localPeek(new DLearnContextPartitionKey(part, learningCtxId, key))); } + /** {@inheritDoc} */ @Override public void remove(String key) { learningCtxCache.remove(new DLearnContextPartitionKey(part, learningCtxId, key)); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java index 51b5902c4447b..e7608ef24b6e0 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java @@ -77,16 +77,17 @@ public LocalDLearnContext(Map learningCtxMap, } /** {@inheritDoc} */ - @Override public > C transform(DLearnContextTransformer transformer) { - UUID newLearningCtxId = UUID.randomUUID(); + @Override public > C transform( + DLearnContextTransformer transformer) { + UUID newLearnCtxId = UUID.randomUUID(); compute((part, partIdx) -> { - DLearnPartitionStorage newStorage = new LocalDLearnPartitionStorage(learningCtxMap, newLearningCtxId, partIdx); + DLearnPartitionStorage newStorage = new LocalDLearnPartitionStorage(learningCtxMap, newLearnCtxId, partIdx); T newPart = transformer.createPartition(newStorage); transformer.transform(part, newPart); }); - DLearnContext newCtx = new LocalDLearnContext<>(learningCtxMap, transformer, newLearningCtxId, partitions); + DLearnContext newCtx = new LocalDLearnContext<>(learningCtxMap, transformer, newLearnCtxId, partitions); return transformer.wrapContext(newCtx); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java index 42108f3b467f1..cf7f981ba617a 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java @@ -21,7 +21,11 @@ import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; /** - * Learning context partition which uses local on-heap hash map to keep data. + * D-learn partition which uses local on-heap hash map to keep data and produced as initial context partition by + * {@link LocalDLearnContextFactory}. + * + * @param type of keys + * @param type of values */ public class LocalDLearnPartition implements AutoCloseable { /** */ @@ -51,4 +55,4 @@ public void setPartData(Map partData) { @Override public void close() { storage.remove(PART_DATA_KEY); } -} \ No newline at end of file +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartitionStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartitionStorage.java index 07759cb3bd6ab..290a540262c5f 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartitionStorage.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartitionStorage.java @@ -22,14 +22,23 @@ import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; +/** + * D-learn partition storage based on on-heap hash map for local processing and tests. Doesn't require Ignite cluster + * to work. + */ public class LocalDLearnPartitionStorage implements DLearnPartitionStorage { - + /** + * Storage. + */ private final Map learningCtxMap; + /** */ private final UUID learningCtxId; + /** */ private final int part; + /** */ public LocalDLearnPartitionStorage( Map learningCtxMap, UUID learningCtxId, int part) { this.learningCtxMap = learningCtxMap; @@ -37,15 +46,18 @@ public LocalDLearnPartitionStorage( this.part = part; } + /** {@inheritDoc} */ @Override public void put(String key, T val) { learningCtxMap.put(new DLearnContextPartitionKey(part, learningCtxId, key), val); } + /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public T get(String key) { - return (T) learningCtxMap.get(new DLearnContextPartitionKey(part, learningCtxId, key)); + return (T)learningCtxMap.get(new DLearnContextPartitionKey(part, learningCtxId, key)); } + /** {@inheritDoc} */ @Override public void remove(String key) { learningCtxMap.remove(new DLearnContextPartitionKey(part, learningCtxId, key)); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/package-info.java index dc961d2b1ebc0..fbab0bf7c3184 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/package-info.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/package-info.java @@ -17,6 +17,6 @@ /** * - * Root package for distributed learning implementations (based on cache, local storage, etc...). + * Root package for underlying d-learn implementations (based on cache, local storage, etc...). */ package org.apache.ignite.ml.dlearn.context; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformer.java index bf2c6229d0a28..05ea3a9f2683a 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformer.java @@ -3,11 +3,32 @@ import org.apache.ignite.ml.dlearn.DLearnContext; import org.apache.ignite.ml.dlearn.DLearnPartitionFactory; -/** */ -public interface DLearnContextTransformer

> extends DLearnPartitionFactory { - /** */ +/** + * Transformer which allow to transform one learning context into another. Transformation mean that new d-learn + * partitions will be created from old partitions and saved in the same underlying storage, but with a new learning + * context id. New partitions will be containing a new data required to provide new API as a new learning context. + * + * All generic transformers are aggregated in {@link DLearnContextTransformers}. + * + * @param

type of an initial partition + * @param type of a new partition + * @param type of a new learning context + */ +public interface DLearnContextTransformer

> + extends DLearnPartitionFactory { + /** + * Copies required data from old partition into new one. All needed transformations are allowed. + * + * @param oldPart old (initial) partition + * @param newPart new partition + */ public void transform(P oldPart, T newPart); - /** */ + /** + * Wraps learning context to provide partition-specific API. + * + * @param ctx context + * @return wrapped context + */ public C wrapContext(DLearnContext ctx); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java index 996338cc7f682..35691f904f35b 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java @@ -1,30 +1,77 @@ package org.apache.ignite.ml.dlearn.context.transformer; +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContextFactory; +import org.apache.ignite.ml.dlearn.context.local.LocalDLearnContextFactory; import org.apache.ignite.ml.dlearn.context.transformer.cache.CacheDatasetDLearnPartitionTransformer; import org.apache.ignite.ml.dlearn.context.transformer.cache.CacheLabeledDatasetDLearnPartitionTransformer; import org.apache.ignite.ml.dlearn.context.transformer.local.LocalDatasetDLearnPartitionTransformer; import org.apache.ignite.ml.dlearn.context.transformer.local.LocalLabeledDatasetDLearnPartitionTransformer; +import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; +import org.apache.ignite.ml.dlearn.dataset.DLearnLabeledDataset; import org.apache.ignite.ml.math.functions.IgniteBiFunction; -/** */ +/** + * Aggregator which allows to find desired transformer from one learning context into another. This class doesn't + * introduce a new functionality, but helps to work efficiently with existing transformers. + */ public class DLearnContextTransformers { - /** */ - public static CacheDatasetDLearnPartitionTransformer cacheToDataset(IgniteBiFunction featureExtractor) { + /** + * Creates a transformer which accepts cache learning context (produced by {@link CacheDLearnContextFactory}) and + * constructs {@link DLearnDataset}. + * + * @param featureExtractor feature extractor + * @param type of keys in cache learning context + * @param type of values in cache learning context + * @return transformer + */ + public static CacheDatasetDLearnPartitionTransformer cacheToDataset( + IgniteBiFunction featureExtractor) { return new CacheDatasetDLearnPartitionTransformer<>(featureExtractor); } - /** */ - public static CacheLabeledDatasetDLearnPartitionTransformer cacheToLabeledDataset(IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + /** + * Creates a transformer which accepts cache learning context (produced by {@link CacheDLearnContextFactory}) and + * constructs {@link DLearnLabeledDataset}. + * + * @param featureExtractor feature extractor + * @param lbExtractor label extractor + * @param type of keys in cache learning context + * @param type of values in cache learning context + * @param type of label + * @return transformer + */ + public static CacheLabeledDatasetDLearnPartitionTransformer cacheToLabeledDataset( + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { return new CacheLabeledDatasetDLearnPartitionTransformer<>(featureExtractor, lbExtractor); } - /** */ - public static LocalDatasetDLearnPartitionTransformer localToDataset(IgniteBiFunction featureExtractor) { + /** + * Creates a transformer which accepts local learning context (produced by {@link LocalDLearnContextFactory}) and + * constructs {@link DLearnDataset}. + * + * @param featureExtractor feature extractor + * @param type of keys in local learning context + * @param type of values in local learning context + * @return transformer + */ + public static LocalDatasetDLearnPartitionTransformer localToDataset( + IgniteBiFunction featureExtractor) { return new LocalDatasetDLearnPartitionTransformer<>(featureExtractor); } - /** */ - public static LocalLabeledDatasetDLearnPartitionTransformer localToLabeledDataset(IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + /** + * Creates a transformer which accepts cache learning context (produced by {@link LocalDLearnContextFactory}) and + * constructs {@link DLearnLabeledDataset}. + * + * @param featureExtractor feature extractor + * @param lbExtractor label extractor + * @param type of keys in local learning context + * @param type of values in local learning context + * @param type of label + * @return transformer + */ + public static LocalLabeledDatasetDLearnPartitionTransformer localToLabeledDataset( + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { return new LocalLabeledDatasetDLearnPartitionTransformer<>(featureExtractor, lbExtractor); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java index 7a57084295f4e..ba6e14c423dfc 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java @@ -26,14 +26,23 @@ import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.ml.dlearn.DLearnContext; import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContextFactory; import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; import org.apache.ignite.ml.dlearn.dataset.part.DLeanDatasetPartition; import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; -/** */ -public class CacheDatasetDLearnPartitionTransformer implements DLearnContextTransformer, DLeanDatasetPartition, DLearnDataset> { +/** + * Creates a transformer which accepts cache learning context (produced by {@link CacheDLearnContextFactory}) and + * constructs {@link DLearnDataset}. + * + * @param type of keys in cache learning context + * @param type of values in cache learning context + */ +public class CacheDatasetDLearnPartitionTransformer + implements DLearnContextTransformer, DLeanDatasetPartition, + DLearnDataset> { /** */ private static final long serialVersionUID = -7398727071330763144L; @@ -45,7 +54,7 @@ public CacheDatasetDLearnPartitionTransformer(IgniteBiFunction f this.featureExtractor = featureExtractor; } - /** */ + /** {@inheritDoc} */ @Override public void transform(CacheDLearnPartition oldPart, DLeanDatasetPartition newPart) { List> partData = queryPartDataIntoList(oldPart); @@ -70,17 +79,22 @@ public CacheDatasetDLearnPartitionTransformer(IgniteBiFunction f newPart.setRows(m); } - /** */ + /** {@inheritDoc} */ @Override public DLearnDataset wrapContext(DLearnContext ctx) { return new DLearnDataset<>(ctx); } - /** */ + /** {@inheritDoc} */ @Override public DLeanDatasetPartition createPartition(DLearnPartitionStorage storage) { return new DLeanDatasetPartition(storage); } - /** */ + /** + * Retrieves local partition data from the cache via {@link ScanQuery} and collects it into list. + * + * @param oldPart partition + * @return list of cache entries + */ private List> queryPartDataIntoList(CacheDLearnPartition oldPart) { List> partData = new ArrayList<>(); for (Cache.Entry entry : queryPartData(oldPart)) @@ -88,7 +102,12 @@ private List> queryPartDataIntoList(CacheDLearnPartition return partData; } - /** */ + /** + * Retrieves local partition data from the cache via {@link ScanQuery} and returns cursor. + * + * @param oldPart partition + * @return cursor + */ private Iterable> queryPartData(CacheDLearnPartition oldPart) { Ignite ignite = Ignition.localIgnite(); IgniteCache upstreamCache = ignite.cache(oldPart.getUpstreamCacheName()); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java index 83a271094d700..a8e75c2a84581 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java @@ -26,14 +26,24 @@ import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.ml.dlearn.DLearnContext; import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContextFactory; import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; import org.apache.ignite.ml.dlearn.dataset.DLearnLabeledDataset; import org.apache.ignite.ml.dlearn.dataset.part.DLearnLabeledDatasetPartition; import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; -/** */ -public class CacheLabeledDatasetDLearnPartitionTransformer implements DLearnContextTransformer, DLearnLabeledDatasetPartition, DLearnLabeledDataset> { +/** + * Creates a transformer which accepts cache learning context (produced by {@link CacheDLearnContextFactory}) and + * constructs {@link DLearnLabeledDataset}. + * + * @param type of keys in cache learning context + * @param type of values in cache learning context + * @param type of label + */ +public class CacheLabeledDatasetDLearnPartitionTransformer + implements DLearnContextTransformer, DLearnLabeledDatasetPartition, + DLearnLabeledDataset> { /** */ private static final long serialVersionUID = 3479218902890029731L; @@ -51,14 +61,14 @@ public CacheLabeledDatasetDLearnPartitionTransformer( this.lbExtractor = lbExtractor; } - /** */ + /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public void transform(CacheDLearnPartition oldPart, DLearnLabeledDatasetPartition newPart) { List> partData = queryPartDataIntoList(oldPart); int m = partData.size(), n = 0; double[] features = null; - L[] labels = (L[]) new Object[m]; + L[] labels = (L[])new Object[m]; for (int i = 0; i < partData.size(); i++) { Cache.Entry entry = partData.get(i); double[] rowFeatures = featureExtractor.apply(entry.getKey(), entry.getValue()); @@ -80,17 +90,22 @@ public CacheLabeledDatasetDLearnPartitionTransformer( newPart.setLabels(labels); } - /** */ + /** {@inheritDoc} */ @Override public DLearnLabeledDataset wrapContext(DLearnContext> ctx) { return new DLearnLabeledDataset<>(ctx); } - /** */ + /** {@inheritDoc} */ @Override public DLearnLabeledDatasetPartition createPartition(DLearnPartitionStorage storage) { return new DLearnLabeledDatasetPartition<>(storage); } - /** */ + /** + * Retrieves local partition data from the cache via {@link ScanQuery} and collects it into list. + * + * @param oldPart partition + * @return list of cache entries + */ private List> queryPartDataIntoList(CacheDLearnPartition oldPart) { List> partData = new ArrayList<>(); for (Cache.Entry entry : queryPartData(oldPart)) @@ -98,7 +113,12 @@ private List> queryPartDataIntoList(CacheDLearnPartition return partData; } - /** */ + /** + * Retrieves local partition data from the cache via {@link ScanQuery} and returns cursor. + * + * @param oldPart partition + * @return cursor + */ private Iterable> queryPartData(CacheDLearnPartition oldPart) { Ignite ignite = Ignition.localIgnite(); IgniteCache upstreamCache = ignite.cache(oldPart.getUpstreamCacheName()); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/package-info.java new file mode 100644 index 0000000000000..31025da3914ab --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Contains transformers which allows to transform cache learning context into some another contexts. + */ +package org.apache.ignite.ml.dlearn.context.transformer.cache; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java index 0684d25865ced..58068177dbf69 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java @@ -22,15 +22,23 @@ import java.util.Map; import org.apache.ignite.ml.dlearn.DLearnContext; import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; +import org.apache.ignite.ml.dlearn.context.local.LocalDLearnContextFactory; import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; import org.apache.ignite.ml.dlearn.dataset.part.DLeanDatasetPartition; import org.apache.ignite.ml.math.functions.IgniteBiFunction; -/** */ +/** + * Creates a transformer which accepts local learning context (produced by {@link LocalDLearnContextFactory}) and + * constructs {@link DLearnDataset}. + * + * @param type of keys in local learning context + * @param type of values in local learning context + */ public class LocalDatasetDLearnPartitionTransformer - implements DLearnContextTransformer, DLeanDatasetPartition, DLearnDataset> { + implements DLearnContextTransformer, DLeanDatasetPartition, + DLearnDataset> { /** */ private static final long serialVersionUID = -7567051002880704559L; @@ -42,7 +50,7 @@ public LocalDatasetDLearnPartitionTransformer(IgniteBiFunction f this.featureExtractor = featureExtractor; } - /** */ + /** {@inheritDoc} */ @Override public void transform(LocalDLearnPartition oldPart, DLeanDatasetPartition newPart) { Map partData = oldPart.getPartData(); if (partData != null && !partData.isEmpty()) { @@ -69,12 +77,12 @@ public LocalDatasetDLearnPartitionTransformer(IgniteBiFunction f } } - /** */ + /** {@inheritDoc} */ @Override public DLearnDataset wrapContext(DLearnContext ctx) { return new DLearnDataset<>(ctx); } - /** */ + /** {@inheritDoc} */ @Override public DLeanDatasetPartition createPartition(DLearnPartitionStorage storage) { return new DLeanDatasetPartition(storage); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java index a6cd558a0e3a8..f0dd76b912847 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java @@ -22,15 +22,24 @@ import java.util.Map; import org.apache.ignite.ml.dlearn.DLearnContext; import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; +import org.apache.ignite.ml.dlearn.context.local.LocalDLearnContextFactory; import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; import org.apache.ignite.ml.dlearn.dataset.DLearnLabeledDataset; import org.apache.ignite.ml.dlearn.dataset.part.DLearnLabeledDatasetPartition; import org.apache.ignite.ml.math.functions.IgniteBiFunction; -/** */ +/** + * Creates a transformer which accepts cache learning context (produced by {@link LocalDLearnContextFactory}) and + * constructs {@link DLearnLabeledDataset}. + * + * @param type of keys in local learning context + * @param type of values in local learning context + * @param type of label + */ public class LocalLabeledDatasetDLearnPartitionTransformer - implements DLearnContextTransformer,DLearnLabeledDatasetPartition, DLearnLabeledDataset> { + implements DLearnContextTransformer, DLearnLabeledDatasetPartition, + DLearnLabeledDataset> { /** */ private static final long serialVersionUID = -8438445094768312331L; @@ -41,12 +50,13 @@ public class LocalLabeledDatasetDLearnPartitionTransformer private final IgniteBiFunction lbExtractor; /** */ - public LocalLabeledDatasetDLearnPartitionTransformer(IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + public LocalLabeledDatasetDLearnPartitionTransformer(IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { this.featureExtractor = featureExtractor; this.lbExtractor = lbExtractor; } - /** */ + /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public void transform(LocalDLearnPartition oldPart, DLearnLabeledDatasetPartition newPart) { Map partData = oldPart.getPartData(); @@ -72,7 +82,7 @@ public LocalLabeledDatasetDLearnPartitionTransformer(IgniteBiFunction wrapContext(DLearnContext> ctx) { return new DLearnLabeledDataset<>(ctx); } - /** */ + /** {@inheritDoc} */ @Override public DLearnLabeledDatasetPartition createPartition(DLearnPartitionStorage storage) { return new DLearnLabeledDatasetPartition<>(storage); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/package-info.java new file mode 100644 index 0000000000000..63b895e7ed420 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Contains transformers which allows to transform local learning context into some another contexts. + */ +package org.apache.ignite.ml.dlearn.context.transformer.local; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/package-info.java index 627432fd801c6..a80edfef99b6c 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/package-info.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/package-info.java @@ -17,6 +17,6 @@ /** * - * Root package for d-learn (distributed learning) framework. + * Contains different transformers which allows to transform one learning context into another. */ package org.apache.ignite.ml.dlearn.context.transformer; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java index 73cc8a90a5ccb..91f928ad23fe4 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java @@ -23,12 +23,23 @@ import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; -/** */ +/** + * Wrapper of a learning context which allows to introduce new context-specific methods based on base {@code compute()} + * functionality. + * + * @param

type of a d-learn partition + */ public class AbstractDLearnContextWrapper

implements DLearnContext

{ - /** */ + /** + * Delegate which actually performs base functions like {@code compute()}, {@code transform()} and {@code close()}. + */ protected final DLearnContext

delegate; - /** */ + /** + * Constructs a new instance of context wrapper which delegates base operations to {@code delegate}. + * + * @param delegate delegate + */ public AbstractDLearnContextWrapper(DLearnContext

delegate) { this.delegate = delegate; } @@ -44,7 +55,8 @@ public AbstractDLearnContextWrapper(DLearnContext

delegate) { } /** {@inheritDoc} */ - @Override public > C transform(DLearnContextTransformer transformer) { + @Override public > C transform( + DLearnContextTransformer transformer) { return delegate.transform(transformer); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnDataset.java index cb385b29b3fd9..9ddd4087b825e 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnDataset.java @@ -21,12 +21,16 @@ import org.apache.ignite.ml.dlearn.DLearnContext; import org.apache.ignite.ml.dlearn.dataset.part.DLeanDatasetPartition; -/** */ +/** + * Dataset provides API to work with dataset. + * + * @param

type of a d-learn partition + */ public class DLearnDataset

extends AbstractDLearnContextWrapper

{ /** */ private static final BLAS blas = BLAS.getInstance(); - /** */ + /** {@inheritDoc} */ public DLearnDataset(DLearnContext

delegate) { super(delegate); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnLabeledDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnLabeledDataset.java index 3a84519639035..d25179b158b00 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnLabeledDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnLabeledDataset.java @@ -20,8 +20,13 @@ import org.apache.ignite.ml.dlearn.DLearnContext; import org.apache.ignite.ml.dlearn.dataset.part.DLearnLabeledDatasetPartition; +/** + * Dataset provides API to work with labeled dataset. + * + * @param type of a label + */ public class DLearnLabeledDataset extends DLearnDataset> { - /** */ + /** {@inheritDoc} */ public DLearnLabeledDataset( DLearnContext> delegate) { super(delegate); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/package-info.java new file mode 100644 index 0000000000000..28b60f7120aea --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Root package for API classes used in algorithms based in d-learn framework. + */ +package org.apache.ignite.ml.dlearn.dataset; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextPartitionKey.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextPartitionKey.java index 8b33f126bdf9d..9f0ae25748133 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextPartitionKey.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextPartitionKey.java @@ -22,7 +22,7 @@ import java.util.UUID; /** - * Key used to identify a-learn partition objects. + * Key used to identify d-learn partition objects. */ public class DLearnContextPartitionKey implements Serializable { /** */ diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java b/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java index a37714e3a573e..87acaa8130667 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java @@ -72,9 +72,7 @@ public void testTrainOnBostonDataset() { // cache with specified transformation (it will be performed locally because partitions are on the same nodes). // In this case for every partition in upstream cache will be created labeled dataset partition and this new // partition will be filled with help of specified feature and label extractors. - - - DLearnLabeledDataset dataset = cacheLearningCtx + DLearnLabeledDataset dataset = cacheLearningCtx .transform( DLearnContextTransformers.cacheToLabeledDataset( (k, v) -> Arrays.copyOfRange(v, 1, v.length), // specify feature extractor @@ -83,21 +81,23 @@ public void testTrainOnBostonDataset() { ); // Calculation of mean value. This calculation will be performed in map-reduce manner. - double[] mean = dataset.mean(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); + double[] mean = dataset.mean(new int[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); System.err.println("Mean values : " + Arrays.toString(mean)); - TestUtils.assertEquals(new double[]{3.59376071e+00, 1.13636364e+01, 1.11367787e+01, 6.91699605e-02, - 5.54695059e-01, 6.28463439e+00, 6.85749012e+01, 3.79504269e+00, - 9.54940711e+00, 4.08237154e+02}, mean, 1e-6); + TestUtils.assertEquals(new double[] { + 3.59376071e+00, 1.13636364e+01, 1.11367787e+01, 6.91699605e-02, + 5.54695059e-01, 6.28463439e+00, 6.85749012e+01, 3.79504269e+00, + 9.54940711e+00, 4.08237154e+02}, mean, 1e-6); // Calculation of standard deviation. This calculation will be performed in map-reduce manner. - double[] std = dataset.std(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); + double[] std = dataset.std(new int[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); System.err.println("Std values : " + Arrays.toString(std)); - TestUtils.assertEquals(new double[]{8.58828355e+00, 2.32993957e+01, 6.85357058e+00, 2.53742935e-01, - 1.15763115e-01, 7.01922514e-01, 2.81210326e+01, 2.10362836e+00, - 8.69865112e+00, 1.68370495e+02}, std, 1e-6); + TestUtils.assertEquals(new double[] { + 8.58828355e+00, 2.32993957e+01, 6.85357058e+00, 2.53742935e-01, + 1.15763115e-01, 7.01922514e-01, 2.81210326e+01, 2.10362836e+00, + 8.69865112e+00, 1.68370495e+02}, std, 1e-6); // Calculation of covariance matrix. This calculation will be performed in map-reduce manner. - double[][] cov = dataset.cov(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); + double[][] cov = dataset.cov(new int[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); System.err.println("Covariance matrix : "); for (double[] row : cov) System.err.println(Arrays.toString(row)); From 146c0785b2c125baf3216c6dd2ad8c38fec9f5a8 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Wed, 17 Jan 2018 18:20:04 +0300 Subject: [PATCH 07/38] IGNITE-7437 Add affinity tests for d-learn context factory. --- .../context/cache/CacheDLearnContext.java | 11 ++ .../cache/CacheDLearnContextFactory.java | 19 ++- .../DLearnPartitionAffinityFunction.java | 59 ++++----- .../context/local/LocalDLearnContext.java | 6 + .../local/LocalDLearnContextFactory.java | 3 + ...acheDatasetDLearnPartitionTransformer.java | 3 +- ...eledDatasetDLearnPartitionTransformer.java | 3 +- ...ocalDatasetDLearnPartitionTransformer.java | 4 + ...eledDatasetDLearnPartitionTransformer.java | 8 +- .../ml/dlearn/dataset/DLearnDataset.java | 92 +++++++++++--- .../java/org/apache/ignite/ml/Playground.java | 22 +++- .../cache/CacheDLearnContextFactoryTest.java | 118 ++++++++++++++++++ .../context/cache/CacheDLearnContextTest.java | 13 +- .../DLearnPartitionAffinityFunctionTest.java | 24 ++++ 14 files changed, 320 insertions(+), 65 deletions(-) create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactoryTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunctionTest.java diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java index f3f07e38d01a8..3bb634512fc1a 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java @@ -72,12 +72,16 @@ public R compute(IgniteBiFunction mapper, IgniteBinaryOperato int[] partitions = affinity.primaryPartitions(locNode); R res = null; + for (int part : partitions) { DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, learningCtxId, part); + P learningCtxPart = partFactory.createPartition(storage); R partRes = mapper.apply(learningCtxPart, part); + res = reducer.apply(res, partRes); } + return res; }); @@ -95,9 +99,12 @@ public R compute(IgniteBiFunction mapper, IgniteBinaryOperato ClusterNode locNode = ignite.cluster().localNode(); int[] partitions = affinity.primaryPartitions(locNode); + for (int part : partitions) { DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, learningCtxId, part); + P learningCtxPart = partFactory.createPartition(storage); + mapper.accept(learningCtxPart, part); } }); @@ -111,7 +118,9 @@ public R compute(IgniteBiFunction mapper, IgniteBinaryOperato compute((part, partIdx) -> { IgniteCache learningCtxCache = ignite.cache(learningCtxCacheName); DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, newLearnCtxId, partIdx); + T newPart = transformer.createPartition(storage); + transformer.transform(part, newPart); }); @@ -123,8 +132,10 @@ public R compute(IgniteBiFunction mapper, IgniteBinaryOperato /** */ private R reduce(Collection results, IgniteBinaryOperator reducer) { R res = null; + for (R partRes : results) res = reducer.apply(res, partRes); + return res; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java index be49fab6d3f30..fdd2b1f73cd72 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java @@ -18,6 +18,7 @@ package org.apache.ignite.ml.dlearn.context.cache; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.UUID; import org.apache.ignite.Ignite; @@ -41,7 +42,7 @@ public class CacheDLearnContextFactory implements DLearnContextFactory upstreamCache) /** {@inheritDoc} */ @Override public CacheDLearnContext> createContext() { CacheConfiguration learningCtxCacheCfg = new CacheConfiguration<>(); - learningCtxCacheCfg.setName(String.format(LEARNING_CONTEXT_CACHE_NAME, upstreamCache, UUID.randomUUID())); + learningCtxCacheCfg.setName(String.format(CONTEXT_CACHE_NAME, upstreamCache.getName(), UUID.randomUUID())); learningCtxCacheCfg.setAffinity(createLearningContextCacheAffinityFunction()); IgniteCache learningCtxCache = ignite.createCache(learningCtxCacheCfg); Affinity affinity = ignite.affinity(upstreamCache.getName()); UUID learningCtxId = UUID.randomUUID(); + for (int partIdx = 0; partIdx < affinity.partitions(); partIdx++) { DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, learningCtxId, partIdx); CacheDLearnPartition part = new CacheDLearnPartition<>(storage); + part.setUpstreamCacheName(upstreamCache.getName()); part.setPart(partIdx); } @@ -87,15 +90,19 @@ private DLearnPartitionAffinityFunction createLearningContextCacheAffinityFuncti // tries to collect partition-to-node map and checks that topology version hasn't been changed during this // process - List initAssignment; + List> initAssignment; long topVer; while (true) { topVer = ignite.cluster().topologyVersion(); initAssignment = new ArrayList<>(affinity.partitions()); + for (int part = 0; part < affinity.partitions(); part++) { - ClusterNode primaryNode = affinity.mapPartitionToNode(part); - initAssignment.add(primaryNode.id()); + Collection nodes = affinity.mapPartitionToPrimaryAndBackups(part); + List nodeIds = new ArrayList<>(nodes.size()); + for (ClusterNode node : nodes) + nodeIds.add(node.id()); + initAssignment.add(nodeIds); } // if topology version changed we need to try again @@ -103,6 +110,6 @@ private DLearnPartitionAffinityFunction createLearningContextCacheAffinityFuncti break; } - return new DLearnPartitionAffinityFunction(initAssignment, topVer, 0); + return new DLearnPartitionAffinityFunction(initAssignment); } } \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunction.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunction.java index 05ea4a1eb5adb..edca19678f5cb 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunction.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunction.java @@ -18,10 +18,10 @@ package org.apache.ignite.ml.dlearn.context.cache.utils; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.UUID; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.cache.affinity.AffinityFunction; @@ -42,29 +42,15 @@ public class DLearnPartitionAffinityFunction implements AffinityFunction { /** * Initial distribution of the partitions (copy of upstream cache partitions distribution). */ - private final List initAssignment; - - /** - * Version of the topology used to collect the {@link #initAssignment}. - */ - private final long initTopVer; - - /** - * Number of partition backups. - */ - private final int backups; + private final List> initAssignment; /** * Creates new instance of d-learn partition affinity function initialized with initial distribution. * * @param initAssignment initial distribution of the partitions (copy of upstream cache partitions distribution) - * @param initTopVer version of the topology used to collect the {@link #initAssignment} - * @param backups number of partition backups */ - public DLearnPartitionAffinityFunction(List initAssignment, long initTopVer, int backups) { + public DLearnPartitionAffinityFunction(List> initAssignment) { this.initAssignment = initAssignment; - this.initTopVer = initTopVer; - this.backups = backups; } /** {@inheritDoc} */ @@ -86,20 +72,37 @@ public DLearnPartitionAffinityFunction(List initAssignment, long initTopVe /** {@inheritDoc} */ @Override public List> assignPartitions(AffinityFunctionContext affCtx) { - long currTopVer = affCtx.currentTopologyVersion().topologyVersion(); List> assignment = new ArrayList<>(initAssignment.size()); - if (currTopVer == initTopVer) { - Map topSnapshotIdx = new HashMap<>(); - for (ClusterNode node : affCtx.currentTopologySnapshot()) - topSnapshotIdx.put(node.id(), node); - for (int part = 0; part < initAssignment.size(); part++) { - UUID partNodeId = initAssignment.get(part); - ClusterNode partNode = topSnapshotIdx.get(partNodeId); - assignment.add(Collections.singletonList(partNode)); + + Map topSnapshotIdx = new HashMap<>(); + List topSnapshot = affCtx.currentTopologySnapshot(); + + for (ClusterNode node : topSnapshot) + topSnapshotIdx.put(node.id(), node); + + for (int part = 0; part < initAssignment.size(); part++) { + List partNodes = new ArrayList<>(1 + affCtx.backups()); + List partNodeIds = initAssignment.get(part); + + // looking for a primary node + ClusterNode primaryPartNode = null; + for (UUID partNodeId : partNodeIds) { + primaryPartNode = topSnapshotIdx.get(partNodeId); + if (primaryPartNode != null) + break; } - return assignment; + + if (primaryPartNode == null) { + Random random = new Random(); + int nodeIdx = random.nextInt(topSnapshot.size()); + primaryPartNode = topSnapshot.get(nodeIdx); + } + partNodes.add(primaryPartNode); + + assignment.add(partNodes); } - throw new IllegalStateException(); + + return assignment; } /** {@inheritDoc} */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java index e7608ef24b6e0..c239bc5f4e58b 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java @@ -60,8 +60,10 @@ public LocalDLearnContext(Map learningCtxMap, R res = null; for (int partIdx = 0; partIdx < partitions; partIdx++) { DLearnPartitionStorage storage = new LocalDLearnPartitionStorage(learningCtxMap, learningCtxId, partIdx); + P part = partFactory.createPartition(storage); R partRes = mapper.apply(part, partIdx); + res = reducer.apply(res, partRes); } return res; @@ -71,7 +73,9 @@ public LocalDLearnContext(Map learningCtxMap, @Override public void compute(IgniteBiConsumer mapper) { for (int partIdx = 0; partIdx < partitions; partIdx++) { DLearnPartitionStorage storage = new LocalDLearnPartitionStorage(learningCtxMap, learningCtxId, partIdx); + P part = partFactory.createPartition(storage); + mapper.accept(part, partIdx); } } @@ -83,7 +87,9 @@ public LocalDLearnContext(Map learningCtxMap, compute((part, partIdx) -> { DLearnPartitionStorage newStorage = new LocalDLearnPartitionStorage(learningCtxMap, newLearnCtxId, partIdx); + T newPart = transformer.createPartition(newStorage); + transformer.transform(part, newPart); }); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java index e935b4726c87b..4bf46b79658d8 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java @@ -57,13 +57,16 @@ public LocalDLearnContextFactory(Map data, int partitions) { // loads data into learning context partitions List keys = new ArrayList<>(data.keySet()); + for (int partIdx = 0; partIdx < partitions; partIdx++) { Map partData = new HashMap<>(); + for (int j = partIdx * partSize; j < (partIdx + 1) * partSize && j < data.size(); j++) partData.put(keys.get(j), data.get(keys.get(j))); DLearnPartitionStorage storage = new LocalDLearnPartitionStorage(learningCtxMap, learningCtxId, partIdx); LocalDLearnPartition part = new LocalDLearnPartition<>(storage); + part.setPartData(partData); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java index ba6e14c423dfc..56e060064d8af 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java @@ -60,6 +60,7 @@ public CacheDatasetDLearnPartitionTransformer(IgniteBiFunction f double[] features = null; int m = partData.size(), n = 0; + for (int i = 0; i < partData.size(); i++) { Cache.Entry entry = partData.get(i); double[] rowFeatures = featureExtractor.apply(entry.getKey(), entry.getValue()); @@ -75,6 +76,7 @@ public CacheDatasetDLearnPartitionTransformer(IgniteBiFunction f for (int j = 0; j < rowFeatures.length; j++) features[j * m + i] = rowFeatures[j]; } + newPart.setFeatures(features); newPart.setRows(m); } @@ -113,7 +115,6 @@ private Iterable> queryPartData(CacheDLearnPartition old IgniteCache upstreamCache = ignite.cache(oldPart.getUpstreamCacheName()); ScanQuery qry = new ScanQuery<>(); - qry.setLocal(true); qry.setPartition(oldPart.getPart()); return upstreamCache.query(qry); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java index a8e75c2a84581..3209a86f1c6b1 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java @@ -69,6 +69,7 @@ public CacheLabeledDatasetDLearnPartitionTransformer( int m = partData.size(), n = 0; double[] features = null; L[] labels = (L[])new Object[m]; + for (int i = 0; i < partData.size(); i++) { Cache.Entry entry = partData.get(i); double[] rowFeatures = featureExtractor.apply(entry.getKey(), entry.getValue()); @@ -85,6 +86,7 @@ public CacheLabeledDatasetDLearnPartitionTransformer( for (int j = 0; j < rowFeatures.length; j++) features[j * m + i] = rowFeatures[j]; } + newPart.setFeatures(features); newPart.setRows(m); newPart.setLabels(labels); @@ -124,7 +126,6 @@ private Iterable> queryPartData(CacheDLearnPartition old IgniteCache upstreamCache = ignite.cache(oldPart.getUpstreamCacheName()); ScanQuery qry = new ScanQuery<>(); - qry.setLocal(true); qry.setPartition(oldPart.getPart()); return upstreamCache.query(qry); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java index 58068177dbf69..5bca2b5c05cf6 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java @@ -53,10 +53,13 @@ public LocalDatasetDLearnPartitionTransformer(IgniteBiFunction f /** {@inheritDoc} */ @Override public void transform(LocalDLearnPartition oldPart, DLeanDatasetPartition newPart) { Map partData = oldPart.getPartData(); + if (partData != null && !partData.isEmpty()) { double[] features = null; int m = partData.size(), n = 0; + List keys = new ArrayList<>(partData.keySet()); + for (int i = 0; i < partData.size(); i++) { K key = keys.get(i); double[] rowFeatures = featureExtractor.apply(key, partData.get(key)); @@ -72,6 +75,7 @@ public LocalDatasetDLearnPartitionTransformer(IgniteBiFunction f for (int j = 0; j < rowFeatures.length; j++) features[j * m + i] = rowFeatures[j]; } + newPart.setFeatures(features); newPart.setRows(m); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java index f0dd76b912847..4c2a239802d5a 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java @@ -60,13 +60,16 @@ public LocalLabeledDatasetDLearnPartitionTransformer(IgniteBiFunction oldPart, DLearnLabeledDatasetPartition newPart) { Map partData = oldPart.getPartData(); + if (partData != null && !partData.isEmpty()) { double[] features = null; int m = partData.size(), n = 0; + List keys = new ArrayList<>(partData.keySet()); + for (int i = 0; i < partData.size(); i++) { K key = keys.get(i); - double[] rowFeatures = featureExtractor.apply(key, partData.get(i)); + double[] rowFeatures = featureExtractor.apply(key, partData.get(key)); if (i == 0) { n = rowFeatures.length; @@ -79,14 +82,17 @@ public LocalLabeledDatasetDLearnPartitionTransformer(IgniteBiFunction delegate) { super(delegate); } - /** */ + /** + * Calculates mean value by given columns. + * + * @param cols columns + * @return mean values + */ public double[] mean(int[] cols) { ValueWithCount res = delegate.compute((part, partIdx) -> { double[] features = part.getFeatures(); int m = part.getRows(); + double[] y = new double[cols.length]; + for (int i = 0; i < cols.length; i++) for (int j = cols[i] * m; j < (cols[i] + 1) * m; j++) y[i] += features[j]; + return new ValueWithCount<>(y, m); - }, (a, b) -> a == null ? b : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); - blas.dscal(res.val.length, 1.0 / res.cnt, res.val, 1); - return res.val; + }, (a, b) -> a == null ? b : b == null ? a : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); + + if (res != null) { + blas.dscal(res.val.length, 1.0 / res.cnt, res.val, 1); + return res.val; + } + + return null; } - /** */ + /** + * Calculates mean value by the given column. + * + * @param col column + * @return mean value + */ public double mean(int col) { return mean(new int[]{col})[0]; } - /** */ + /** + * Calculates standard deviation by given columns. + * + * @param cols columns + * @return standard deviations + */ public double[] std(int[] cols) { double[] mean = mean(cols); ValueWithCount res = delegate.compute(part -> { double[] features = part.getFeatures(); int m = part.getRows(); + double[] y = new double[cols.length]; + for (int i = 0; i < cols.length; i++) for (int j = cols[i] * m; j < (cols[i] + 1) * m; j++) y[i] += Math.pow(features[j] - mean[cols[i]], 2); + return new ValueWithCount<>(y, m); - }, (a, b) -> a == null ? b : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); - blas.dscal(res.val.length, 1.0 / res.cnt, res.val, 1); - for (int i = 0; i < res.val.length; i++) - res.val[i] = Math.sqrt(res.val[i]); - return res.val; + }, (a, b) -> a == null ? b : b == null ? a : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); + + if (res != null) { + blas.dscal(res.val.length, 1.0 / res.cnt, res.val, 1); + for (int i = 0; i < res.val.length; i++) + res.val[i] = Math.sqrt(res.val[i]); + return res.val; + } + + return null; } - /** */ + /** + * Calculates standard deviation by the given column. + * + * @param col column + * @return standard deviation + */ public double std(int col) { return std(new int[]{col})[0]; } - /** */ + /** + * Calculates covariance matrix by given columns. + * + * @param cols columns + * @return covariance matrix + */ public double[][] cov(int[] cols) { double[] mean = mean(cols); ValueWithCount res = delegate.compute(part -> { double[] features = part.getFeatures(); int m = part.getRows(); + double[][] y = new double[cols.length][cols.length]; + for (int i = 0; i < cols.length; i++) for (int j = 0; j < cols.length; j++) { int firstCol = cols[i]; int secondCol = cols[j]; + for (int k = 0; k < m; k++) { double firstVal = features[m * firstCol + k]; double secondVal = features[m * secondCol + k]; y[i][j] += ((firstVal - mean[firstCol]) * (secondVal - mean[secondCol])); } } + return new ValueWithCount<>(y, m); - }, (a, b) -> a == null ? b : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); - return scale(res.val, 1.0 / res.cnt); + }, (a, b) -> a == null ? b : b == null ? a : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); + + return res != null ? scale(res.val, 1.0 / res.cnt) : null; } - /** */ + /** + * Calculates correlation matrix by given columns. + * + * @param cols columns + * @return correlation matrix + */ public double[][] corr(int[] cols) { double[][] cov = cov(cols); double[] std = std(cols); + for (int i = 0; i < cov.length; i++) for (int j = 0; j < cov[0].length; j++) cov[i][j] /= (std[i]*std[j]); + return cov; } @@ -114,20 +167,25 @@ public double[][] corr(int[] cols) { private static double[] sum(double[] a, double[] b) { for (int i = 0; i < a.length; i++) a[i] += b[i]; + return a; } + /** */ private static double[][] sum(double[][] a, double[][] b) { for (int i = 0; i < a.length; i++) for (int j = 0; j < a[i].length; j++) a[i][j] += b[i][j]; + return a; } + /** */ private static double[][] scale(double[][] a, double alpha) { for (int i = 0; i < a.length; i++) for (int j = 0; j < a[i].length; j++) a[i][j] *= alpha; + return a; } @@ -140,7 +198,7 @@ private static class ValueWithCount { private final int cnt; /** */ - public ValueWithCount(V val, int cnt) { + ValueWithCount(V val, int cnt) { this.val = val; this.cnt = cnt; } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java b/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java index 87acaa8130667..895a4cca25c25 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java @@ -24,7 +24,7 @@ import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.dlearn.DLearnContext; +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContext; import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContextFactory; import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformers; @@ -34,7 +34,7 @@ /** */ public class Playground extends GridCommonAbstractTest { /** Number of nodes in grid */ - private static final int NODE_COUNT = 4; + private static final int NODE_COUNT = 2; /** */ private Ignite ignite; @@ -61,11 +61,21 @@ public class Playground extends GridCommonAbstractTest { } /** */ - public void testTrainOnBostonDataset() { + public void testTrainOnBostonDataset() throws Exception { +// ignite.events().localListen(evt -> { +// System.err.println("EVT PART LOADED : " + evt); +// return true; +// }, EventType.EVT_CACHE_REBALANCE_PART_LOADED); +// +// ignite.events().localListen(evt -> { +// System.err.println("EVT PART UNLOADED : " + evt); +// return true; +// }, EventType.EVT_CACHE_REBALANCE_PART_UNLOADED); + IgniteCache bostonDataset = loadDataset(); // Initialization of d-learn context, after this step context cache will be created with partitions placed on // the same nodes as the upstream Ignite Cache (in this case bostonDataset). - DLearnContext> cacheLearningCtx = + CacheDLearnContext> cacheLearningCtx = new CacheDLearnContextFactory<>(ignite, bostonDataset).createContext(); // Loading of the d-learn context. During this step data will be transferred from the upstream cache to context @@ -107,8 +117,8 @@ public void testTrainOnBostonDataset() { private IgniteCache loadDataset() { CacheConfiguration cc = new CacheConfiguration<>(); cc.setName("BOSTON_DATASET"); - cc.setAffinity(new RendezvousAffinityFunction(true, 10)); - IgniteCache cache = ignite.createCache(cc); + cc.setAffinity(new RendezvousAffinityFunction(false, 10)); + IgniteCache cache = ignite.getOrCreateCache(cc); int nvars = 13; Scanner scanner = new Scanner(this.getClass().getClassLoader().getResourceAsStream("datasets/regression/boston.csv")); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactoryTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactoryTest.java new file mode 100644 index 0000000000000..03edc6eb4c752 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactoryTest.java @@ -0,0 +1,118 @@ +/* + * 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.ml.dlearn.context.cache; + +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests for {@link CacheDLearnContextFactory}. + */ +public class CacheDLearnContextFactoryTest extends GridCommonAbstractTest { + /** Number of nodes in grid */ + private static final int NODE_COUNT = 10; + + /** */ + private Ignite ignite; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + for (int i = 1; i <= NODE_COUNT; i++) + startGrid(i); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() { + stopAllGrids(); + } + + /** + * {@inheritDoc} + */ + @Override protected void beforeTest() throws Exception { + /* Grid instance. */ + ignite = grid(NODE_COUNT); + ignite.configuration().setPeerClassLoadingEnabled(true); + IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); + } + + /** + * Test that partitions of the context created on top of the dataset are placed on the same nodes as initial + * dataset (number of partitions are less than number of nodes). + */ + public void testAffinityWithNumberOfPartitionsLessThanNodes() { + testAffinity(generateTestData(5, 10), 5); + } + + /** + * Test that partitions of the context created on top of the dataset are placed on the same nodes as initial + * dataset (number of partitions are greater than number of nodes). + */ + public void testAffinityWithNumberOfPartitionsGreaterThanNodes() { + testAffinity(generateTestData(50, 10), 50); + } + + /** */ + private void testAffinity(IgniteCache data, int partitions) { + CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); + + CacheDLearnContext> ctx = ctxFactory.createContext(); + + Affinity dataAffinity = ignite.affinity(data.getName()); + Affinity ctxAffinity = ignite.affinity(ctx.getLearningCtxCacheName()); + + assertEquals(partitions, dataAffinity.partitions()); + assertEquals(partitions, ctxAffinity.partitions()); + + for (int part = 0; part < partitions; part++) { + ClusterNode dataPartPrimaryNode = dataAffinity.mapPartitionToNode(part); + ClusterNode ctxPartPrimaryNode = ctxAffinity.mapPartitionToNode(part); + + assertNotNull(dataPartPrimaryNode); + assertNotNull(ctxPartPrimaryNode); + assertEquals(dataPartPrimaryNode, ctxPartPrimaryNode); + } + } + + /** + * Generates Ignite Cache with data for tests. + * + * @return Ignite Cache with data for tests + */ + private IgniteCache generateTestData(int partitions, int backups) { + CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); + + cacheConfiguration.setName(UUID.randomUUID().toString()); + cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, partitions)); + cacheConfiguration.setBackups(backups); + + IgniteCache cache = ignite.createCache(cacheConfiguration); + + for (int i = 0; i < partitions * 10; i++) + cache.put(i, String.valueOf(i)); + + return cache; + } +} \ No newline at end of file diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java index c5e0c88f2ee20..ed22603016a25 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java @@ -61,7 +61,7 @@ public class CacheDLearnContextTest extends GridCommonAbstractTest { /** */ public void testClose() { - IgniteCache data = generateTestData(); + IgniteCache data = generateTestData(2, 0); CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); CacheDLearnContext> ctx = ctxFactory.createContext(); @@ -78,7 +78,7 @@ public void testClose() { /** */ public void testCloseDerivativeContext() { - IgniteCache data = generateTestData(); + IgniteCache data = generateTestData(2, 0); CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); CacheDLearnContext> ctx = ctxFactory.createContext(); @@ -105,7 +105,7 @@ public void testCloseDerivativeContext() { /** */ public void testCloseBaseContext() { - IgniteCache data = generateTestData(); + IgniteCache data = generateTestData(2, 0); CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); CacheDLearnContext> ctx = ctxFactory.createContext(); @@ -135,12 +135,15 @@ public void testCloseBaseContext() { * * @return Ignite Cache with data for tests */ - private IgniteCache generateTestData() { + private IgniteCache generateTestData(int partitions, int backups) { CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); + cacheConfiguration.setName(UUID.randomUUID().toString()); - cacheConfiguration.setAffinity(new RendezvousAffinityFunction(true, 2)); + cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, partitions)); + cacheConfiguration.setBackups(backups); IgniteCache cache = ignite.createCache(cacheConfiguration); + cache.put(1, "TEST1"); cache.put(2, "TEST2"); cache.put(3, "TEST3"); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunctionTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunctionTest.java new file mode 100644 index 0000000000000..840ecc1323f07 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunctionTest.java @@ -0,0 +1,24 @@ +/* + * 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.ml.dlearn.context.cache.utils; + +/** + * Tests for {@link DLearnPartitionAffinityFunction}. + */ +public class DLearnPartitionAffinityFunctionTest { +} From 6c4f191081019c91a4b78f40bdf6f52a9f46eb47 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Wed, 17 Jan 2018 19:07:08 +0300 Subject: [PATCH 08/38] IGNITE-7437 Add dataset loading examples (cache and local) for d-learn. --- .../ml/dlearn/CacheDatasetExample.java | 124 ++++++++++++++++ .../ml/dlearn/LocalDatasetExample.java | 122 +++++++++++++++ .../examples/ml/dlearn/package-info.java | 22 +++ .../java/org/apache/ignite/ml/Playground.java | 140 ------------------ 4 files changed, 268 insertions(+), 140 deletions(-) create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlearn/CacheDatasetExample.java create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlearn/LocalDatasetExample.java create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlearn/package-info.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/Playground.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/CacheDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/CacheDatasetExample.java new file mode 100644 index 0000000000000..a54fa082b9eca --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/CacheDatasetExample.java @@ -0,0 +1,124 @@ +/* + * 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.examples.ml.dlearn; + +import java.util.Arrays; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContext; +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContextFactory; +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformers; +import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; + +/** + * How to create a d-learn dataset from an existing Ignite Cache? + */ +public class CacheDatasetExample { + /** Run example. */ + public static void main(String[] args) { + try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { + System.out.println(">>> D-Learn Dataset example started."); + + IgniteCache persons = createCache(ignite); + + // Initialization of the d-learn context. After this step context cache will be created with partitions + // placed on the same nodes as the upstream Ignite Cache. + CacheDLearnContext> ctx = + new CacheDLearnContextFactory<>(ignite, persons).createContext(); + + // Loading of the d-learn context. During this step data will be transferred from the upstream cache to + // context cache with specified transformation (it will be performed locally because partitions are on the + // same nodes). In this case for every partition in upstream cache will be created labeled dataset partition + // and this new partition will be filled with help of specified feature and label extractors. + DLearnDataset dataset = ctx.transform(DLearnContextTransformers.cacheToDataset((k, v) -> { + double[] row = new double[2]; + row[0] = v.age; + row[1] = v.salary; + return row; + })); + + // Calculation of the mean value. This calculation will be performed in map-reduce manner. + double[] mean = dataset.mean(new int[] {0, 1}); + System.out.println("Mean \n\t" + Arrays.toString(mean)); + + // Calculation of the standard deviation. This calculation will be performed in map-reduce manner. + double[] std = dataset.std(new int[] {0, 1}); + System.out.println("Standard deviation \n\t" + Arrays.toString(std)); + + // Calculation of the covariance matrix. This calculation will be performed in map-reduce manner. + double[][] cov = dataset.cov(new int[] {0, 1}); + System.out.println("Covariance matrix "); + for (double[] row : cov) + System.out.println("\t" + Arrays.toString(row)); + + // Calculation of the correlation matrix. This calculation will be performed in map-reduce manner. + double[][] corr = dataset.corr(new int[] {0, 1}); + System.out.println("Correlation matrix "); + for (double[] row : corr) + System.out.println("\t" + Arrays.toString(row)); + + System.out.println(">>> D-Learn Dataset example completed."); + } + } + + /** */ + private static IgniteCache createCache(Ignite ignite) { + IgniteCache persons = ignite.createCache("PERSONS"); + persons.put(1, new Person("Mike", 42, 10000)); + persons.put(2, new Person("John", 32, 64000)); + persons.put(3, new Person("George", 53, 120000)); + persons.put(4, new Person("Karl", 24, 70000)); + return persons; + } + + /** */ + private static class Person { + /** */ + private final String name; + + /** */ + private final double age; + + /** */ + private final double salary; + + /** */ + public Person(String name, double age, double salary) { + this.name = name; + this.age = age; + this.salary = salary; + } + + /** */ + public String getName() { + return name; + } + + /** */ + public double getAge() { + return age; + } + + /** */ + public double getSalary() { + return salary; + } + } +} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/LocalDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/LocalDatasetExample.java new file mode 100644 index 0000000000000..f1e389a93d035 --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/LocalDatasetExample.java @@ -0,0 +1,122 @@ +/* + * 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.examples.ml.dlearn; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.Ignite; +import org.apache.ignite.Ignition; +import org.apache.ignite.ml.dlearn.context.local.LocalDLearnContext; +import org.apache.ignite.ml.dlearn.context.local.LocalDLearnContextFactory; +import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformers; +import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; + +/** + * How to create a d-learn dataset from an existing local data? + */ +public class LocalDatasetExample { + /** Run example. */ + public static void main(String[] args) { + try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { + System.out.println(">>> D-Learn Dataset example started."); + + Map persons = createCache(ignite); + + // Initialization of the d-learn context. This context uses local hash map as an underlying storage. + LocalDLearnContext> ctx = + new LocalDLearnContextFactory<>(persons, 2).createContext(); + + // Loading of the d-learn context. During this step data will be transferred from the generic local + // partition to dataset partition. + DLearnDataset dataset = ctx.transform(DLearnContextTransformers.localToDataset((k, v) -> { + double[] row = new double[2]; + row[0] = v.age; + row[1] = v.salary; + return row; + })); + + // Calculation of the mean value. This calculation will be performed in map-reduce manner. + double[] mean = dataset.mean(new int[] {0, 1}); + System.out.println("Mean \n\t" + Arrays.toString(mean)); + + // Calculation of the standard deviation. This calculation will be performed in map-reduce manner. + double[] std = dataset.std(new int[] {0, 1}); + System.out.println("Standard deviation \n\t" + Arrays.toString(std)); + + // Calculation of the covariance matrix. This calculation will be performed in map-reduce manner. + double[][] cov = dataset.cov(new int[] {0, 1}); + System.out.println("Covariance matrix "); + for (double[] row : cov) + System.out.println("\t" + Arrays.toString(row)); + + // Calculation of the correlation matrix. This calculation will be performed in map-reduce manner. + double[][] corr = dataset.corr(new int[] {0, 1}); + System.out.println("Correlation matrix "); + for (double[] row : corr) + System.out.println("\t" + Arrays.toString(row)); + + System.out.println(">>> D-Learn Dataset example completed."); + } + } + + /** */ + private static Map createCache(Ignite ignite) { + Map persons = new HashMap<>(); + persons.put(1, new Person("Mike", 42, 10000)); + persons.put(2, new Person("John", 32, 64000)); + persons.put(3, new Person("George", 53, 120000)); + persons.put(4, new Person("Karl", 24, 70000)); + return persons; + } + + /** */ + private static class Person { + /** */ + private final String name; + + /** */ + private final double age; + + /** */ + private final double salary; + + /** */ + public Person(String name, double age, double salary) { + this.name = name; + this.age = age; + this.salary = salary; + } + + /** */ + public String getName() { + return name; + } + + /** */ + public double getAge() { + return age; + } + + /** */ + public double getSalary() { + return salary; + } + } +} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/package-info.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/package-info.java new file mode 100644 index 0000000000000..e3c92ec0a1b7f --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Machine learning d-learn framework examples. + */ +package org.apache.ignite.examples.ml.dlearn; \ No newline at end of file diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java b/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java deleted file mode 100644 index 895a4cca25c25..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/Playground.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * 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.ml; - -import java.util.Arrays; -import java.util.Scanner; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContext; -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContextFactory; -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformers; -import org.apache.ignite.ml.dlearn.dataset.DLearnLabeledDataset; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -/** */ -public class Playground extends GridCommonAbstractTest { - /** Number of nodes in grid */ - private static final int NODE_COUNT = 2; - - /** */ - private Ignite ignite; - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - for (int i = 1; i <= NODE_COUNT; i++) - startGrid(i); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() { - stopAllGrids(); - } - - /** - * {@inheritDoc} - */ - @Override protected void beforeTest() throws Exception { - /* Grid instance. */ - ignite = grid(NODE_COUNT); - ignite.configuration().setPeerClassLoadingEnabled(true); - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - } - - /** */ - public void testTrainOnBostonDataset() throws Exception { -// ignite.events().localListen(evt -> { -// System.err.println("EVT PART LOADED : " + evt); -// return true; -// }, EventType.EVT_CACHE_REBALANCE_PART_LOADED); -// -// ignite.events().localListen(evt -> { -// System.err.println("EVT PART UNLOADED : " + evt); -// return true; -// }, EventType.EVT_CACHE_REBALANCE_PART_UNLOADED); - - IgniteCache bostonDataset = loadDataset(); - // Initialization of d-learn context, after this step context cache will be created with partitions placed on - // the same nodes as the upstream Ignite Cache (in this case bostonDataset). - CacheDLearnContext> cacheLearningCtx = - new CacheDLearnContextFactory<>(ignite, bostonDataset).createContext(); - - // Loading of the d-learn context. During this step data will be transferred from the upstream cache to context - // cache with specified transformation (it will be performed locally because partitions are on the same nodes). - // In this case for every partition in upstream cache will be created labeled dataset partition and this new - // partition will be filled with help of specified feature and label extractors. - DLearnLabeledDataset dataset = cacheLearningCtx - .transform( - DLearnContextTransformers.cacheToLabeledDataset( - (k, v) -> Arrays.copyOfRange(v, 1, v.length), // specify feature extractor - (k, v) -> v[0] // specify label extractor - ) - ); - - // Calculation of mean value. This calculation will be performed in map-reduce manner. - double[] mean = dataset.mean(new int[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); - System.err.println("Mean values : " + Arrays.toString(mean)); - TestUtils.assertEquals(new double[] { - 3.59376071e+00, 1.13636364e+01, 1.11367787e+01, 6.91699605e-02, - 5.54695059e-01, 6.28463439e+00, 6.85749012e+01, 3.79504269e+00, - 9.54940711e+00, 4.08237154e+02}, mean, 1e-6); - - // Calculation of standard deviation. This calculation will be performed in map-reduce manner. - double[] std = dataset.std(new int[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); - System.err.println("Std values : " + Arrays.toString(std)); - TestUtils.assertEquals(new double[] { - 8.58828355e+00, 2.32993957e+01, 6.85357058e+00, 2.53742935e-01, - 1.15763115e-01, 7.01922514e-01, 2.81210326e+01, 2.10362836e+00, - 8.69865112e+00, 1.68370495e+02}, std, 1e-6); - - // Calculation of covariance matrix. This calculation will be performed in map-reduce manner. - double[][] cov = dataset.cov(new int[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); - System.err.println("Covariance matrix : "); - for (double[] row : cov) - System.err.println(Arrays.toString(row)); - } - - /** */ - private IgniteCache loadDataset() { - CacheConfiguration cc = new CacheConfiguration<>(); - cc.setName("BOSTON_DATASET"); - cc.setAffinity(new RendezvousAffinityFunction(false, 10)); - IgniteCache cache = ignite.getOrCreateCache(cc); - - int nvars = 13; - Scanner scanner = new Scanner(this.getClass().getClassLoader().getResourceAsStream("datasets/regression/boston.csv")); - int i = 0; - while (scanner.hasNextLine()) { - String row = scanner.nextLine(); - int j = 0; - double[] r = new double[nvars + 1]; - for (String feature : row.split(",")) { - r[j] = Double.parseDouble(feature); - j++; - } - cache.put(i, r); - i++; - } - - return cache; - } -} From 21b648b182305c34a061e55d7d2b287165c362eb Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Wed, 17 Jan 2018 19:40:12 +0300 Subject: [PATCH 09/38] IGNITE-7437 Add dataset transformation example for d-learn. --- .../ml/dlearn/CacheDatasetExample.java | 4 +- .../ml/dlearn/LocalDatasetExample.java | 4 +- .../ml/dlearn/TransformerExample.java | 231 ++++++++++++++++++ 3 files changed, 235 insertions(+), 4 deletions(-) create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlearn/TransformerExample.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/CacheDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/CacheDatasetExample.java index a54fa082b9eca..056d6c2d25546 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/CacheDatasetExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/CacheDatasetExample.java @@ -34,7 +34,7 @@ public class CacheDatasetExample { /** Run example. */ public static void main(String[] args) { try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { - System.out.println(">>> D-Learn Dataset example started."); + System.out.println(">>> D-Learn Cache Dataset example started."); IgniteCache persons = createCache(ignite); @@ -74,7 +74,7 @@ public static void main(String[] args) { for (double[] row : corr) System.out.println("\t" + Arrays.toString(row)); - System.out.println(">>> D-Learn Dataset example completed."); + System.out.println(">>> D-Learn Cache Dataset example completed."); } } diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/LocalDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/LocalDatasetExample.java index f1e389a93d035..1e91ff0b620cb 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/LocalDatasetExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/LocalDatasetExample.java @@ -35,7 +35,7 @@ public class LocalDatasetExample { /** Run example. */ public static void main(String[] args) { try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { - System.out.println(">>> D-Learn Dataset example started."); + System.out.println(">>> D-Learn Local Dataset example started."); Map persons = createCache(ignite); @@ -72,7 +72,7 @@ public static void main(String[] args) { for (double[] row : corr) System.out.println("\t" + Arrays.toString(row)); - System.out.println(">>> D-Learn Dataset example completed."); + System.out.println(">>> D-Learn Local Dataset example completed."); } } diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/TransformerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/TransformerExample.java new file mode 100644 index 0000000000000..13e94043b9efa --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/TransformerExample.java @@ -0,0 +1,231 @@ +/* + * 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.examples.ml.dlearn; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.ml.dlearn.DLearnContext; +import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContext; +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContextFactory; +import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformers; +import org.apache.ignite.ml.dlearn.dataset.AbstractDLearnContextWrapper; +import org.apache.ignite.ml.dlearn.dataset.DLearnLabeledDataset; +import org.apache.ignite.ml.dlearn.dataset.part.DLearnLabeledDatasetPartition; + +/** + * How to transform dataset into algorithm-specific view? + */ +public class TransformerExample { + /** Run example. */ + public static void main(String[] args) { + try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { + System.out.println(">>> D-Learn Dataset example started."); + + IgniteCache persons = createCache(ignite); + + // Initialization of the d-learn context. After this step context cache will be created with partitions + // placed on the same nodes as the upstream Ignite Cache. + CacheDLearnContext> ctx = + new CacheDLearnContextFactory<>(ignite, persons).createContext(); + + // Loading of the d-learn context. During this step data will be transferred from the upstream cache to + // context cache with specified transformation (it will be performed locally because partitions are on the + // same nodes). In this case for every partition in upstream cache will be created labeled dataset partition + // and this new partition will be filled with help of specified feature and label extractors. + DLearnLabeledDataset dataset = ctx.transform(DLearnContextTransformers.cacheToLabeledDataset( + (k, v) -> new double[]{ v.age }, + (k, v) -> v.salary + )); + + AlgorithmSpecificContext algorithmSpecificCtx = dataset.transform(new AlgorithmSpecificTransformer()); + System.out.println("Result: " + algorithmSpecificCtx.solve()); + } + } + + /** + * Algorithm-specific partition. + */ + private static class AlgorithmSpecificPartition implements AutoCloseable { + /** */ + private static final String MATRIX_A_KEY = "matrix_a"; + + /** */ + private static final String VECTOR_B_KEY = "vector_b"; + + /** */ + private static final String ROWS_KEY = "rows"; + + /** */ + private final DLearnPartitionStorage storage; + + /** */ + public AlgorithmSpecificPartition(DLearnPartitionStorage storage) { + this.storage = storage; + } + + /** */ + public double[] getMatrixA() { + return storage.get(MATRIX_A_KEY); + } + + /** */ + public void setMatrixA(double[] matrixA) { + storage.put(MATRIX_A_KEY, matrixA); + } + + /** */ + public Integer getRows() { + return storage.get(ROWS_KEY); + } + + /** */ + public void setRows(int rows) { + storage.put(ROWS_KEY, rows); + } + + /** */ + public double[] getVectorB() { + return storage.get(VECTOR_B_KEY); + } + + /** */ + public void setVectorB(double[] vectorB) { + storage.put(VECTOR_B_KEY, vectorB); + } + + /** */ + @Override public void close() { + storage.remove(MATRIX_A_KEY); + storage.remove(VECTOR_B_KEY); + storage.remove(ROWS_KEY); + } + } + + /** + * Algorithm-specific representation of the context. + */ + private static class AlgorithmSpecificContext extends AbstractDLearnContextWrapper { + /** */ + public AlgorithmSpecificContext( + DLearnContext delegate) { + super(delegate); + } + + /** */ + public double solve() { + return compute(part -> { + double[] matrixA = part.getMatrixA(); + double[] vectorB = part.getVectorB(); + Integer rows = part.getRows(); + // do something to solve... + return 42.0; + }, (a, b) -> a == null ? b : a + b); + } + } + + /** + * Algorithm-specific transformer. + */ + private static class AlgorithmSpecificTransformer implements DLearnContextTransformer, + AlgorithmSpecificPartition, AlgorithmSpecificContext> { + /** */ + private static final long serialVersionUID = 2109144841306143061L; + + /** */ + @Override public void transform(DLearnLabeledDatasetPartition oldPart, + AlgorithmSpecificPartition newPart) { + double[] features = oldPart.getFeatures(); + + if (features != null) { + Object[] labels = oldPart.getLabels(); + int rows = oldPart.getRows(); + + double[] aMatrix = new double[features.length + rows]; + double[] bVector = new double[rows]; + + for (int i = 0; i < rows; i++) { + aMatrix[i] = 1.0; + bVector[i] = (Double) labels[i]; + } + + newPart.setMatrixA(aMatrix); + newPart.setVectorB(bVector); + newPart.setRows(rows); + } + } + + /** */ + @Override public AlgorithmSpecificContext wrapContext( + DLearnContext ctx) { + return new AlgorithmSpecificContext(ctx); + } + + /** */ + @Override public AlgorithmSpecificPartition createPartition(DLearnPartitionStorage storage) { + return new AlgorithmSpecificPartition(storage); + } + } + + /** */ + private static IgniteCache createCache(Ignite ignite) { + IgniteCache persons = ignite.createCache("PERSONS"); + persons.put(1, new Person("Mike", 42, 10000)); + persons.put(2, new Person("John", 32, 64000)); + persons.put(3, new Person("George", 53, 120000)); + persons.put(4, new Person("Karl", 24, 70000)); + return persons; + } + + /** */ + private static class Person { + /** */ + private final String name; + + /** */ + private final double age; + + /** */ + private final double salary; + + /** */ + public Person(String name, double age, double salary) { + this.name = name; + this.age = age; + this.salary = salary; + } + + /** */ + public String getName() { + return name; + } + + /** */ + public double getAge() { + return age; + } + + /** */ + public double getSalary() { + return salary; + } + } +} From 87194ba42c7dd3250e0dac30dd11bb1a1283022c Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Fri, 19 Jan 2018 19:21:59 +0300 Subject: [PATCH 10/38] IGNITE-7437 Use affinityCall/affinityRun in cache d-learn context implementation to guarantee partitions reservation during computation. --- .../ignite/ml/dlearn/DLearnContext.java | 32 +- .../context/cache/CacheDLearnContext.java | 87 +++-- .../cache/CacheDLearnContextFactory.java | 4 +- .../context/local/LocalDLearnContext.java | 5 +- ...acheDatasetDLearnPartitionTransformer.java | 13 +- .../dataset/AbstractDLearnContextWrapper.java | 5 +- .../context/cache/CacheDLearnContextTest.java | 367 +++++++++++++++++- 7 files changed, 465 insertions(+), 48 deletions(-) diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java index edefbbed3aef0..2bb21329ad251 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java @@ -39,10 +39,40 @@ public interface DLearnContext

extends AutoCloseable { * * @param mapper mapper function applied on every partition * @param reducer reducer of the results + * @param identity identity value * @param result type * @return final reduced result */ - public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer); + public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer, R identity); + + /** + * Computes a given function on every d-learn partition in current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param mapper mapper function applied on every partition + * @param reducer reducer of the results + * @param result type + * @return final reduced result + */ + default public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer) { + return compute(mapper, reducer, null); + } + + /** + * Computes a given function on every d-learn partition in current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param mapper mapper function applied on every partition + * @param reducer reducer of the results + * @param identity identity value + * @param result type + * @return final reduced result + */ + default public R compute(IgniteFunction mapper, IgniteBinaryOperator reducer, R identity) { + return compute((part, partIdx) -> mapper.apply(part), reducer, identity); + } /** * Computes a given function on every d-learn partition in current learning context independently and then reduces diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java index 3bb634512fc1a..4263ca01b8704 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java @@ -17,18 +17,21 @@ package org.apache.ignite.ml.dlearn.context.cache; +import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; +import java.util.List; import java.util.UUID; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.affinity.Affinity; import org.apache.ignite.cluster.ClusterGroup; -import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.ml.dlearn.DLearnContext; import org.apache.ignite.ml.dlearn.DLearnPartitionFactory; import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; -import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; +import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; import org.apache.ignite.ml.math.functions.IgniteBiConsumer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; @@ -51,63 +54,80 @@ public class CacheDLearnContext

implements DLearnContex /** */ private final UUID learningCtxId; + /** */ + private final Collection cacheNames; + /** */ public CacheDLearnContext(Ignite ignite, String learningCtxCacheName, DLearnPartitionFactory

partFactory, - UUID learningCtxId) { + UUID learningCtxId, Collection cacheNames) { this.ignite = ignite; this.learningCtxCacheName = learningCtxCacheName; this.partFactory = partFactory; this.learningCtxId = learningCtxId; + this.cacheNames = cacheNames; } /** {@inheritDoc} */ - public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer) { + public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer, R identity) { ClusterGroup clusterGrp = ignite.cluster().forDataNodes(learningCtxCacheName); - Collection results = ignite.compute(clusterGrp).broadcast(() -> { - IgniteCache learningCtxCache = ignite.cache(learningCtxCacheName); + IgniteCache learningCtxCache = ignite.cache(learningCtxCacheName); - Affinity affinity = ignite.affinity(learningCtxCacheName); - ClusterNode locNode = ignite.cluster().localNode(); + Affinity affinity = ignite.affinity(learningCtxCacheName); - int[] partitions = affinity.primaryPartitions(locNode); - R res = null; + List> futures = new ArrayList<>(affinity.partitions()); - for (int part : partitions) { - DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, learningCtxId, part); + for (int part = 0; part < affinity.partitions(); part++) { + int currPart = part; + IgniteFuture fut = ignite.compute(clusterGrp).affinityCallAsync( + cacheNames, + part, + () -> { + DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, learningCtxId, currPart); - P learningCtxPart = partFactory.createPartition(storage); - R partRes = mapper.apply(learningCtxPart, part); + P learningCtxPart = partFactory.createPartition(storage); - res = reducer.apply(res, partRes); - } + return mapper.apply(learningCtxPart, currPart); + } + ); + futures.add(fut); + } - return res; - }); + List results = new ArrayList<>(affinity.partitions()); + for (IgniteFuture future : futures) + results.add(future.get()); - return reduce(results, reducer); + return reduce(results, reducer, identity); } /** {@inheritDoc} */ @Override public void compute(IgniteBiConsumer mapper) { ClusterGroup clusterGrp = ignite.cluster().forDataNodes(learningCtxCacheName); - ignite.compute(clusterGrp).broadcast(() -> { - IgniteCache learningCtxCache = ignite.cache(learningCtxCacheName); + IgniteCache learningCtxCache = ignite.cache(learningCtxCacheName); - Affinity affinity = ignite.affinity(learningCtxCacheName); - ClusterNode locNode = ignite.cluster().localNode(); + Affinity affinity = ignite.affinity(learningCtxCacheName); - int[] partitions = affinity.primaryPartitions(locNode); + List> futures = new ArrayList<>(affinity.partitions()); - for (int part : partitions) { - DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, learningCtxId, part); + for (int part = 0; part < affinity.partitions(); part++) { + int currPart = part; + IgniteFuture fut = ignite.compute(clusterGrp).affinityRunAsync( + cacheNames, + part, + () -> { + DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, learningCtxId, currPart); - P learningCtxPart = partFactory.createPartition(storage); + P learningCtxPart = partFactory.createPartition(storage); - mapper.accept(learningCtxPart, part); - } - }); + mapper.accept(learningCtxPart, currPart); + } + ); + futures.add(fut); + } + + for (IgniteFuture future : futures) + future.get(); } /** {@inheritDoc} */ @@ -124,14 +144,15 @@ public R compute(IgniteBiFunction mapper, IgniteBinaryOperato transformer.transform(part, newPart); }); - DLearnContext newCtx = new CacheDLearnContext<>(ignite, learningCtxCacheName, transformer, newLearnCtxId); + DLearnContext newCtx = new CacheDLearnContext<>(ignite, learningCtxCacheName, transformer, newLearnCtxId, + Collections.singletonList(learningCtxCacheName)); return transformer.wrapContext(newCtx); } /** */ - private R reduce(Collection results, IgniteBinaryOperator reducer) { - R res = null; + private R reduce(Collection results, IgniteBinaryOperator reducer, R identity) { + R res = identity; for (R partRes : results) res = reducer.apply(res, partRes); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java index fdd2b1f73cd72..f1d592a8195bc 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java @@ -18,6 +18,7 @@ package org.apache.ignite.ml.dlearn.context.cache; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.UUID; @@ -75,7 +76,8 @@ public CacheDLearnContextFactory(Ignite ignite, IgniteCache upstreamCache) part.setPart(partIdx); } - return new CacheDLearnContext<>(ignite, learningCtxCache.getName(), CacheDLearnPartition::new, learningCtxId); + return new CacheDLearnContext<>(ignite, learningCtxCache.getName(), CacheDLearnPartition::new, learningCtxId, + Arrays.asList(upstreamCache.getName(), learningCtxCache.getName())); } /** diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java index c239bc5f4e58b..614afac0a3155 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java @@ -56,8 +56,9 @@ public LocalDLearnContext(Map learningCtxMap, } /** {@inheritDoc} */ - @Override public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer) { - R res = null; + @Override public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer, + R identity) { + R res = identity; for (int partIdx = 0; partIdx < partitions; partIdx++) { DLearnPartitionStorage storage = new LocalDLearnPartitionStorage(learningCtxMap, learningCtxId, partIdx); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java index 56e060064d8af..86ff7720c5cf2 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java @@ -23,14 +23,15 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.Ignition; +import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.ml.dlearn.DLearnContext; import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContextFactory; import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; +import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; import org.apache.ignite.ml.dlearn.dataset.part.DLeanDatasetPartition; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; /** @@ -99,9 +100,11 @@ public CacheDatasetDLearnPartitionTransformer(IgniteBiFunction f */ private List> queryPartDataIntoList(CacheDLearnPartition oldPart) { List> partData = new ArrayList<>(); - for (Cache.Entry entry : queryPartData(oldPart)) - partData.add(entry); - return partData; + try (QueryCursor> cursor = queryPartData(oldPart)) { + for (Cache.Entry entry : cursor) + partData.add(entry); + return partData; + } } /** @@ -110,7 +113,7 @@ private List> queryPartDataIntoList(CacheDLearnPartition * @param oldPart partition * @return cursor */ - private Iterable> queryPartData(CacheDLearnPartition oldPart) { + private QueryCursor> queryPartData(CacheDLearnPartition oldPart) { Ignite ignite = Ignition.localIgnite(); IgniteCache upstreamCache = ignite.cache(oldPart.getUpstreamCacheName()); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java index 91f928ad23fe4..9c249ebdf8d62 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java @@ -45,8 +45,9 @@ public AbstractDLearnContextWrapper(DLearnContext

delegate) { } /** {@inheritDoc} */ - @Override public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer) { - return delegate.compute(mapper, reducer); + @Override public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer, + R identity) { + return delegate.compute(mapper, reducer, identity); } /** {@inheritDoc} */ diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java index ed22603016a25..926b087ef0a34 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java @@ -17,12 +17,27 @@ package org.apache.ignite.ml.dlearn.context.cache; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.UUID; +import java.util.concurrent.locks.LockSupport; import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteAtomicLong; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteLock; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology; import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformers; import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; @@ -130,6 +145,352 @@ public void testCloseBaseContext() { assertEquals(0, learningCtxCache.size()); } + /** + * Tests that partitions of upstream cache and context cache are reserved during computations on base learning + * context. Reservation means that partitions won't be unloaded from the node before computation will be completed. + */ + public void testPartitionExchangeDuringComputeWithReturnCallOnBaseContext() { + int partitions = 4; + + IgniteCache data = generateTestData(4, 0); + + // create learning context + CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); + CacheDLearnContext> ctx = ctxFactory.createContext(); + + assertTrue("Before computation all partitions should not be reserved", + areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); + + IgniteAtomicLong numOfStartedComputations = ignite.atomicLong(UUID.randomUUID().toString(), 0, true); + + IgniteLock computationsLock = ignite.reentrantLock(UUID.randomUUID().toString(), false, true, true); + + // lock computations lock to stop computations in the middle + computationsLock.lock(); + + try { + new Thread(() -> ctx.compute((part, partIndex) -> { + // track number of started computations + numOfStartedComputations.incrementAndGet(); + computationsLock.lock(); + computationsLock.unlock(); + return 42; + }, (a, b) -> 42)).start(); + // wait all computations to start + + while (numOfStartedComputations.get() < partitions) {} + + assertTrue("During computation all partitions should be reserved", + areAllPartitionsReserved(data.getName(), ctx.getLearningCtxCacheName())); + } + finally { + computationsLock.unlock(); + } + + assertTrue("All partitions should be released", + areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); + } + + /** + * Tests that partitions of upstream cache and context cache are reserved during computations on base learning + * context. Reservation means that partitions won't be unloaded from the node before computation will be completed. + */ + public void testPartitionExchangeDuringComputeCallOnBaseContext() { + int partitions = 4; + + IgniteCache data = generateTestData(4, 0); + + // create learning context + CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); + CacheDLearnContext> ctx = ctxFactory.createContext(); + + assertTrue("Before computation all partitions should not be reserved", + areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); + + IgniteAtomicLong numOfStartedComputations = ignite.atomicLong(UUID.randomUUID().toString(), 0, true); + + IgniteLock computationsLock = ignite.reentrantLock(UUID.randomUUID().toString(), false, true, true); + + // lock computations lock to stop computations in the middle + computationsLock.lock(); + + try { + new Thread(() -> ctx.compute((part, partIndex) -> { + // track number of started computations + numOfStartedComputations.incrementAndGet(); + computationsLock.lock(); + computationsLock.unlock(); + })).start(); + // wait all computations to start + + while (numOfStartedComputations.get() < partitions) {} + + assertTrue("During computation all partitions should be reserved", + areAllPartitionsReserved(data.getName(), ctx.getLearningCtxCacheName())); + } + finally { + computationsLock.unlock(); + } + + assertTrue("All partitions should be released", + areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); + } + + /** + * Tests that only partitions of context cache are reserved during computations on base learning + * context. Reservation means that partitions won't be unloaded from the node before computation will be completed. + * Partitions of the upstream cache should not be reserved. + */ + public void testPartitionExchangeDuringComputeWithReturnCallOnDerivativeContext() throws InterruptedException { + int partitions = 4; + + IgniteCache data = generateTestData(4, 0); + + // create learning context + CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); + CacheDLearnContext> ctx = ctxFactory.createContext(); + + DLearnDataset dataset = ctx.transform(DLearnContextTransformers.cacheToDataset( (k, v) -> new double[10] )); + + assertTrue("Before computation all partitions should not be reserved", + areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); + + IgniteAtomicLong numOfStartedComputations = ignite.atomicLong(UUID.randomUUID().toString(), 0, true); + + IgniteLock computationsLock = ignite.reentrantLock(UUID.randomUUID().toString(), false, true, true); + + // lock computations lock to stop computations in the middle + computationsLock.lock(); + + try { + new Thread(() -> dataset.compute((part, partIndex) -> { + // track number of started computations + numOfStartedComputations.incrementAndGet(); + computationsLock.lock(); + computationsLock.unlock(); + return 42; + }, (a, b) -> 42)).start(); + // wait all computations to start + + while (numOfStartedComputations.get() < partitions) {} + + assertTrue("During computation all partitions of the context cache should be reserved", + areAllPartitionsReserved(ctx.getLearningCtxCacheName())); + assertTrue("During computation all partitions of the data cache should be reserved", + areAllPartitionsNotReserved(data.getName())); + } + finally { + computationsLock.unlock(); + } + + assertTrue("All partitions should be released", + areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); + } + + /** + * Tests that only partitions of context cache are reserved during computations on base learning + * context. Reservation means that partitions won't be unloaded from the node before computation will be completed. + * Partitions of the upstream cache should not be reserved. + */ + public void testPartitionExchangeDuringComputeCallOnDerivativeContext() throws InterruptedException { + int partitions = 4; + + IgniteCache data = generateTestData(4, 0); + + // create learning context + CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); + CacheDLearnContext> ctx = ctxFactory.createContext(); + + DLearnDataset dataset = ctx.transform(DLearnContextTransformers.cacheToDataset( (k, v) -> new double[10] )); + + assertTrue("Before computation all partitions should not be reserved", + areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); + + IgniteAtomicLong numOfStartedComputations = ignite.atomicLong(UUID.randomUUID().toString(), 0, true); + + IgniteLock computationsLock = ignite.reentrantLock(UUID.randomUUID().toString(), false, true, true); + + // lock computations lock to stop computations in the middle + computationsLock.lock(); + + try { + new Thread(() -> dataset.compute((part, partIndex) -> { + // track number of started computations + numOfStartedComputations.incrementAndGet(); + computationsLock.lock(); + computationsLock.unlock(); + })).start(); + // wait all computations to start + + while (numOfStartedComputations.get() < partitions) {} + + assertTrue("During computation all partitions of the context cache should be reserved", + areAllPartitionsReserved(ctx.getLearningCtxCacheName())); + assertTrue("During computation all partitions of the data cache should be reserved", + areAllPartitionsNotReserved(data.getName())); + } + finally { + computationsLock.unlock(); + } + + assertTrue("All partitions should be released", + areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); + } + + /** + * Checks that all partitions of all specified caches are not reserved. + * + * @param cacheNames cache names to be checked + * @return {@code true} if all partitions are not reserved, otherwise {@code false} + */ + private boolean areAllPartitionsNotReserved(String... cacheNames) { + return checkAllPartitions(partition -> partition.reservations() == 0, cacheNames); + } + + /** + * Checks that all partitions of all specified caches not reserved. + * + * @param cacheNames cache names to be checked + * @return {@code true} if all partitions are reserved, otherwise {@code false} + */ + private boolean areAllPartitionsReserved(String... cacheNames) { + return checkAllPartitions(partition -> partition.reservations() != 0, cacheNames); + } + + /** + * Checks that all partitions of all specified caches satisfies the given predicate. + * + * @param pred predicate + * @param cacheNames cache names + * @return {@code true} if all partitions satisfies the given predicate + */ + private boolean checkAllPartitions(IgnitePredicate pred, String... cacheNames) { + boolean flag = false; + long checkingStartTs = System.currentTimeMillis(); + + while (!flag && (System.currentTimeMillis() - checkingStartTs) < 30_000) { + LockSupport.parkNanos(200 * 1000 * 1000); + flag = true; + + for (String cacheName : cacheNames) { + IgniteClusterPartitionsState state = IgniteClusterPartitionsState.getCurrentState(cacheName); + + for (IgniteInstancePartitionsState instanceState : state.instances.values()) + for (GridDhtLocalPartition partition : instanceState.parts) + if (partition != null) + flag &= pred.apply(partition); + } + } + + return flag; + } + + /** + * Aggregated data about cache partitions in Ignite cluster. + */ + private static class IgniteClusterPartitionsState { + /** */ + private final String cacheName; + + /** */ + private final Map instances; + + /** */ + static IgniteClusterPartitionsState getCurrentState(String cacheName) { + Map instances = new HashMap<>(); + + for (Ignite ignite : G.allGrids()) { + IgniteKernal igniteKernal = (IgniteKernal) ignite; + IgniteCacheProxy cache = igniteKernal.context().cache().jcache(cacheName); + + GridDhtCacheAdapter dht = dht(cache); + + GridDhtPartitionTopology top = dht.topology(); + + AffinityTopologyVersion topVer = dht.context().shared().exchange().readyAffinityVersion(); + List parts = new ArrayList<>(); + for (int p = 0; p < cache.context().config().getAffinity().partitions(); p++) { + GridDhtLocalPartition part = top.localPartition(p, AffinityTopologyVersion.NONE, false); + parts.add(part); + } + instances.put(ignite.cluster().localNode().id(), new IgniteInstancePartitionsState(topVer, parts)); + } + + return new IgniteClusterPartitionsState(cacheName, instances); + } + + /** */ + IgniteClusterPartitionsState(String cacheName, + Map instances) { + this.cacheName = cacheName; + this.instances = instances; + } + + /** */ + @Override public String toString() { + StringBuilder builder = new StringBuilder(); + builder.append("Cache ").append(cacheName).append(" is in following state:").append("\n"); + for (Map.Entry e : instances.entrySet()) { + UUID instanceId = e.getKey(); + IgniteInstancePartitionsState instanceState = e.getValue(); + builder.append("\n\t") + .append("Node ") + .append(instanceId) + .append(" with topology version [") + .append(instanceState.topVer.topologyVersion()) + .append(", ") + .append(instanceState.topVer.minorTopologyVersion()) + .append("] contains following partitions:") + .append("\n\n"); + builder.append("\t\t-----------------------------------------------------------------------------------------------------\n"); + builder.append("\t\t| ID | STATE | RELOAD | RESERVATIONS | SHOULD BE RENTING | PRIMARY | DATA STORE SIZE |\n"); + builder.append("\t\t-----------------------------------------------------------------------------------------------------\n"); + for (GridDhtLocalPartition partition : instanceState.parts) + if (partition != null) { + builder.append("\t\t") + .append(String.format("| %3d |", partition.id())) + .append(String.format(" %7s |", partition.state())) + .append(String.format(" %7s |", partition.reload())) + .append(String.format(" %13s |", partition.reservations())) + .append(String.format(" %18s |", partition.shouldBeRenting())) + .append(String.format(" %8s |", partition.primary(instanceState.topVer))) + .append(String.format(" %16d |", partition.dataStore().fullSize())) + .append("\n"); + builder.append("\t\t-----------------------------------------------------------------------------------------------------\n"); + } + } + return builder.toString(); + } + } + + /** + * Aggregated data about cache partitions in Ignite instance. + */ + private static class IgniteInstancePartitionsState { + /** */ + private final AffinityTopologyVersion topVer; + + /** */ + private final List parts; + + /** */ + IgniteInstancePartitionsState(AffinityTopologyVersion topVer, + List parts) { + this.topVer = topVer; + this.parts = parts; + } + + /** */ + public AffinityTopologyVersion getTopVer() { + return topVer; + } + + /** */ + public List getParts() { + return parts; + } + } + /** * Generates Ignite Cache with data for tests. * @@ -144,10 +505,8 @@ private IgniteCache generateTestData(int partitions, int backup IgniteCache cache = ignite.createCache(cacheConfiguration); - cache.put(1, "TEST1"); - cache.put(2, "TEST2"); - cache.put(3, "TEST3"); - cache.put(4, "TEST4"); + for (int i = 0; i < 1000; i++) + cache.put(i, "TEST" + i); return cache; } From a553f12f3fb0c3b64470d2e3002a5eb49013614c Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Fri, 19 Jan 2018 19:32:10 +0300 Subject: [PATCH 11/38] IGNITE-7437 Reformat code. --- .../context/cache/CacheDLearnContextTest.java | 30 ++++++++++++------- 1 file changed, 19 insertions(+), 11 deletions(-) diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java index 926b087ef0a34..698f3e26f5d52 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java @@ -178,7 +178,8 @@ public void testPartitionExchangeDuringComputeWithReturnCallOnBaseContext() { }, (a, b) -> 42)).start(); // wait all computations to start - while (numOfStartedComputations.get() < partitions) {} + while (numOfStartedComputations.get() < partitions) { + } assertTrue("During computation all partitions should be reserved", areAllPartitionsReserved(data.getName(), ctx.getLearningCtxCacheName())); @@ -223,7 +224,8 @@ public void testPartitionExchangeDuringComputeCallOnBaseContext() { })).start(); // wait all computations to start - while (numOfStartedComputations.get() < partitions) {} + while (numOfStartedComputations.get() < partitions) { + } assertTrue("During computation all partitions should be reserved", areAllPartitionsReserved(data.getName(), ctx.getLearningCtxCacheName())); @@ -250,7 +252,7 @@ public void testPartitionExchangeDuringComputeWithReturnCallOnDerivativeContext( CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); CacheDLearnContext> ctx = ctxFactory.createContext(); - DLearnDataset dataset = ctx.transform(DLearnContextTransformers.cacheToDataset( (k, v) -> new double[10] )); + DLearnDataset dataset = ctx.transform(DLearnContextTransformers.cacheToDataset((k, v) -> new double[10])); assertTrue("Before computation all partitions should not be reserved", areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); @@ -272,7 +274,8 @@ public void testPartitionExchangeDuringComputeWithReturnCallOnDerivativeContext( }, (a, b) -> 42)).start(); // wait all computations to start - while (numOfStartedComputations.get() < partitions) {} + while (numOfStartedComputations.get() < partitions) { + } assertTrue("During computation all partitions of the context cache should be reserved", areAllPartitionsReserved(ctx.getLearningCtxCacheName())); @@ -301,7 +304,7 @@ public void testPartitionExchangeDuringComputeCallOnDerivativeContext() throws I CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); CacheDLearnContext> ctx = ctxFactory.createContext(); - DLearnDataset dataset = ctx.transform(DLearnContextTransformers.cacheToDataset( (k, v) -> new double[10] )); + DLearnDataset dataset = ctx.transform(DLearnContextTransformers.cacheToDataset((k, v) -> new double[10])); assertTrue("Before computation all partitions should not be reserved", areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); @@ -322,7 +325,8 @@ public void testPartitionExchangeDuringComputeCallOnDerivativeContext() throws I })).start(); // wait all computations to start - while (numOfStartedComputations.get() < partitions) {} + while (numOfStartedComputations.get() < partitions) { + } assertTrue("During computation all partitions of the context cache should be reserved", areAllPartitionsReserved(ctx.getLearningCtxCacheName())); @@ -400,7 +404,7 @@ static IgniteClusterPartitionsState getCurrentState(String cacheName) { Map instances = new HashMap<>(); for (Ignite ignite : G.allGrids()) { - IgniteKernal igniteKernal = (IgniteKernal) ignite; + IgniteKernal igniteKernal = (IgniteKernal)ignite; IgniteCacheProxy cache = igniteKernal.context().cache().jcache(cacheName); GridDhtCacheAdapter dht = dht(cache); @@ -442,9 +446,12 @@ static IgniteClusterPartitionsState getCurrentState(String cacheName) { .append(instanceState.topVer.minorTopologyVersion()) .append("] contains following partitions:") .append("\n\n"); - builder.append("\t\t-----------------------------------------------------------------------------------------------------\n"); - builder.append("\t\t| ID | STATE | RELOAD | RESERVATIONS | SHOULD BE RENTING | PRIMARY | DATA STORE SIZE |\n"); - builder.append("\t\t-----------------------------------------------------------------------------------------------------\n"); + builder.append("\t\t---------------------------------------------------------------------------------"); + builder.append("--------------------\n"); + builder.append("\t\t| ID | STATE | RELOAD | RESERVATIONS | SHOULD BE RENTING | PRIMARY |"); + builder.append(" DATA STORE SIZE |\n"); + builder.append("\t\t---------------------------------------------------------------------------------"); + builder.append("--------------------\n"); for (GridDhtLocalPartition partition : instanceState.parts) if (partition != null) { builder.append("\t\t") @@ -456,7 +463,8 @@ static IgniteClusterPartitionsState getCurrentState(String cacheName) { .append(String.format(" %8s |", partition.primary(instanceState.topVer))) .append(String.format(" %16d |", partition.dataStore().fullSize())) .append("\n"); - builder.append("\t\t-----------------------------------------------------------------------------------------------------\n"); + builder.append("\t\t-------------------------------------------------------------------------"); + builder.append("----------------------------\n"); } } return builder.toString(); From 03a6bfe1b47a259e1cf400762dca427ab4ecf928 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Fri, 19 Jan 2018 20:32:49 +0300 Subject: [PATCH 12/38] IGNITE-7437 Update Javadoc. --- .../context/cache/CacheDLearnContext.java | 36 ++++++++++++++----- .../cache/CacheDLearnContextFactory.java | 2 +- 2 files changed, 29 insertions(+), 9 deletions(-) diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java index 4263ca01b8704..db86c890fce32 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java @@ -42,22 +42,30 @@ * @param

type of learning context partition */ public class CacheDLearnContext

implements DLearnContext

{ - /** */ + /** Ignite instance. */ private final Ignite ignite; - /** */ + /** Learning context cache name. */ private final String learningCtxCacheName; - /** */ + /** Partition factory. */ private final DLearnPartitionFactory

partFactory; - /** */ + /** Learning context id. */ private final UUID learningCtxId; - /** */ + /** Names of caches which partitions are reserved during computations. */ private final Collection cacheNames; - /** */ + /** + * Constructs a new instance of cache learning context. + * + * @param ignite Ignite instance + * @param learningCtxCacheName learning context cache name + * @param partFactory partition factory + * @param learningCtxId learning context id + * @param cacheNames names of caches which partitions are reserved during computations + */ public CacheDLearnContext(Ignite ignite, String learningCtxCacheName, DLearnPartitionFactory

partFactory, UUID learningCtxId, Collection cacheNames) { this.ignite = ignite; @@ -150,7 +158,15 @@ public R compute(IgniteBiFunction mapper, IgniteBinaryOperato return transformer.wrapContext(newCtx); } - /** */ + /** + * Reduces results into a single final result. + * + * @param results results + * @param reducer reducer function + * @param identity identity + * @param type of result + * @return single final result + */ private R reduce(Collection results, IgniteBinaryOperator reducer, R identity) { R res = identity; @@ -165,7 +181,11 @@ private R reduce(Collection results, IgniteBinaryOperator reducer, R i compute(this::closePartition); } - /** */ + /** + * Closes partition. + * + * @param part partition to be closed + */ private void closePartition(P part) { try { part.close(); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java index f1d592a8195bc..1ae98b841734c 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java @@ -42,7 +42,7 @@ public class CacheDLearnContextFactory implements DLearnContextFactory Date: Mon, 22 Jan 2018 11:51:37 +0300 Subject: [PATCH 13/38] IGNITE-7437 Update Javadoc. --- .../cache/CacheDLearnContextFactory.java | 11 ++++-- .../context/cache/CacheDLearnPartition.java | 12 ++++--- .../cache/CacheDLearnPartitionStorage.java | 34 ++++++++++++++----- .../context/local/LocalDLearnContext.java | 23 +++++++++---- .../local/LocalDLearnContextFactory.java | 11 ++++-- .../context/local/LocalDLearnPartition.java | 10 ++++-- .../local/LocalDLearnPartitionStorage.java | 16 +++++---- ...acheDatasetDLearnPartitionTransformer.java | 8 +++-- ...eledDatasetDLearnPartitionTransformer.java | 11 ++++-- ...ocalDatasetDLearnPartitionTransformer.java | 8 +++-- ...eledDatasetDLearnPartitionTransformer.java | 11 ++++-- .../ml/dlearn/dataset/DLearnDataset.java | 2 +- .../dataset/part/DLeanDatasetPartition.java | 12 ++++--- .../part/DLearnLabeledDatasetPartition.java | 10 ++++-- .../utils/DLearnContextPartitionKey.java | 20 +++++------ 15 files changed, 138 insertions(+), 61 deletions(-) diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java index 1ae98b841734c..62108784cbee4 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java @@ -45,13 +45,18 @@ public class CacheDLearnContextFactory implements DLearnContextFactory upstreamCache; - /** */ + /** + * Constructs a new instance of cache learning context factory. + * + * @param ignite Ignite instance + * @param upstreamCache upstream cache + */ public CacheDLearnContextFactory(Ignite ignite, IgniteCache upstreamCache) { this.ignite = ignite; this.upstreamCache = upstreamCache; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java index 0aace852bd5a5..1de06134613fc 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java @@ -27,16 +27,20 @@ * @param type of values */ public class CacheDLearnPartition implements AutoCloseable { - /** */ + /** Key of the upstream cache name values. */ private static final String UPSTREAM_CACHE_NAME_KEY = "upstream_cache_name"; - /** */ + /** Key of the partition value. */ private static final String PART_KEY = "part"; - /** */ + /** Partition storage. */ private final DLearnPartitionStorage storage; - /** */ + /** + * Constructs a new instance of cache learning partition. + * + * @param storage partition storage + */ public CacheDLearnPartition(DLearnPartitionStorage storage) { this.storage = storage; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartitionStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartitionStorage.java index d3c65d841adbe..c113167293735 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartitionStorage.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartitionStorage.java @@ -31,18 +31,22 @@ * D-learn partition storage based on Ignite cache. */ public class CacheDLearnPartitionStorage implements DLearnPartitionStorage { - /** - * Storage. - */ + /** Learning context physical storage. */ private final IgniteCache learningCtxCache; - /** */ + /** Learning context id. */ private final UUID learningCtxId; - /** */ + /** Partition index. */ private final int part; - /** */ + /** + * Constructs a new instance of cache learning partition storage. + * + * @param learningCtxCache learning context physical storage + * @param learningCtxId learning context id + * @param part partition index + */ public CacheDLearnPartitionStorage(IgniteCache learningCtxCache, UUID learningCtxId, int part) { this.learningCtxCache = learningCtxCache; @@ -66,8 +70,16 @@ public CacheDLearnPartitionStorage(IgniteCache type of learning context partition */ public class LocalDLearnContext

implements DLearnContext

{ - /** */ + /** Physical storage of the learning context data. */ private final Map learningCtxMap; - /** */ + /** Learning context partition factory. */ private final DLearnPartitionFactory

partFactory; - /** */ + /** Learning context id. */ private final UUID learningCtxId; - /** */ + /** Number of partitions. */ private final int partitions; - /** */ + /** + * Constructs a new instance of local learning context. + * + * @param learningCtxMap physical storage of the learning context data + * @param partFactory learning context partition factory + * @param learningCtxId learning context id + * @param partitions number of partitions + */ public LocalDLearnContext(Map learningCtxMap, DLearnPartitionFactory

partFactory, UUID learningCtxId, int partitions) { this.learningCtxMap = learningCtxMap; @@ -104,7 +111,11 @@ public LocalDLearnContext(Map learningCtxMap, compute(this::closePartition); } - /** */ + /** + * Closes partition. + * + * @param part partition to be closed + */ private void closePartition(P part) { try { part.close(); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java index 4bf46b79658d8..c3e4de4929bd6 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java @@ -35,13 +35,18 @@ public class LocalDLearnContextFactory implements DLearnContextFactory data; - /** */ + /** Number of partitions. */ private final int partitions; - /** */ + /** + * Constructs a new instance of learning context factory. + * + * @param data upstream data + * @param partitions number of partitions + */ public LocalDLearnContextFactory(Map data, int partitions) { this.data = data; this.partitions = partitions; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java index cf7f981ba617a..c6673e93df742 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java @@ -28,13 +28,17 @@ * @param type of values */ public class LocalDLearnPartition implements AutoCloseable { - /** */ + /** Key of partition data values. */ private static final String PART_DATA_KEY = "part_data"; - /** */ + /** Partition storage. */ private final DLearnPartitionStorage storage; - /** */ + /** + * Constructs a new instance of local learning partition. + * + * @param storage partition storage + */ public LocalDLearnPartition(DLearnPartitionStorage storage) { this.storage = storage; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartitionStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartitionStorage.java index 290a540262c5f..175e5fe1bd091 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartitionStorage.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartitionStorage.java @@ -27,18 +27,22 @@ * to work. */ public class LocalDLearnPartitionStorage implements DLearnPartitionStorage { - /** - * Storage. - */ + /** Learning context physical storage. */ private final Map learningCtxMap; - /** */ + /** Learning context id. */ private final UUID learningCtxId; - /** */ + /** Partition index. */ private final int part; - /** */ + /** + * Constructs a new instance of local learning partition storage. + * + * @param learningCtxMap learning context physical storage + * @param learningCtxId learning context id + * @param part partition index + */ public LocalDLearnPartitionStorage( Map learningCtxMap, UUID learningCtxId, int part) { this.learningCtxMap = learningCtxMap; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java index 86ff7720c5cf2..53a4192cddbfb 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java @@ -47,10 +47,14 @@ public class CacheDatasetDLearnPartitionTransformer /** */ private static final long serialVersionUID = -7398727071330763144L; - /** */ + /** Feature extractor. */ private final IgniteBiFunction featureExtractor; - /** */ + /** + * Constructs a new instance of cache to dataset partition transformer. + * + * @param featureExtractor feature extractor + */ public CacheDatasetDLearnPartitionTransformer(IgniteBiFunction featureExtractor) { this.featureExtractor = featureExtractor; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java index 3209a86f1c6b1..720fceb3fa22b 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java @@ -47,13 +47,18 @@ public class CacheLabeledDatasetDLearnPartitionTransformer /** */ private static final long serialVersionUID = 3479218902890029731L; - /** */ + /** Feature extractor. */ private final IgniteBiFunction featureExtractor; - /** */ + /** Label extractor. */ private final IgniteBiFunction lbExtractor; - /** */ + /** + * Constructs a new instance of cache to labeled dataset partition transformer. + * + * @param featureExtractor feature extractor + * @param lbExtractor label extractor + */ public CacheLabeledDatasetDLearnPartitionTransformer( IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java index 5bca2b5c05cf6..1c70a2b38b05a 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java @@ -42,10 +42,14 @@ public class LocalDatasetDLearnPartitionTransformer /** */ private static final long serialVersionUID = -7567051002880704559L; - /** */ + /** Feature extractor. */ private final IgniteBiFunction featureExtractor; - /** */ + /** + * Constructs new instance of local to dataset partition transformer. + * + * @param featureExtractor feature extractor + */ public LocalDatasetDLearnPartitionTransformer(IgniteBiFunction featureExtractor) { this.featureExtractor = featureExtractor; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java index 4c2a239802d5a..7bd41b1bc9eb7 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java @@ -43,13 +43,18 @@ public class LocalLabeledDatasetDLearnPartitionTransformer /** */ private static final long serialVersionUID = -8438445094768312331L; - /** */ + /** Feature extractor. */ private final IgniteBiFunction featureExtractor; - /** */ + /** Label extractor. */ private final IgniteBiFunction lbExtractor; - /** */ + /** + * Creates new instance of local to labeled dataset transformer. + * + * @param featureExtractor feature extractor + * @param lbExtractor label extractor + */ public LocalLabeledDatasetDLearnPartitionTransformer(IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { this.featureExtractor = featureExtractor; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnDataset.java index 83677d9429cde..03b2ae9d35431 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnDataset.java @@ -27,7 +27,7 @@ * @param

type of a d-learn partition */ public class DLearnDataset

extends AbstractDLearnContextWrapper

{ - /** */ + /** BLAS (Basic Linear Algebra Subprograms) instance. */ private static final BLAS blas = BLAS.getInstance(); /** {@inheritDoc} */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLeanDatasetPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLeanDatasetPartition.java index e0cc753af6c2e..5270298427a66 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLeanDatasetPartition.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLeanDatasetPartition.java @@ -23,16 +23,20 @@ * Interface which provides simple dataset API which allows to get or set an underlying feature matrix in flat format. */ public class DLeanDatasetPartition implements AutoCloseable { - /** */ + /** Key of feature values. */ private static final String FEATURES_KEY = "features"; - /** */ + /** Key of row values. */ private static final String ROWS_KEY = "rows"; - /** */ + /** Learning partition storage. */ private final DLearnPartitionStorage storage; - /** */ + /** + * Creates a new instance of learning dataset partition. + * + * @param storage learning partition storage + */ public DLeanDatasetPartition(DLearnPartitionStorage storage) { this.storage = storage; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLearnLabeledDatasetPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLearnLabeledDatasetPartition.java index 86b930e3a7d30..7bac8cc036bb4 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLearnLabeledDatasetPartition.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLearnLabeledDatasetPartition.java @@ -24,13 +24,17 @@ * and vector of labels. */ public class DLearnLabeledDatasetPartition extends DLeanDatasetPartition { - /** */ + /** Key of label values. */ private static final String LABELS_KEY = "labels"; - /** */ + /** Learning partition storage. */ private final DLearnPartitionStorage storage; - /** */ + /** + * Constructs a new instance of labeled dataset partition. + * + * @param storage learning partition storage. + */ public DLearnLabeledDatasetPartition(DLearnPartitionStorage storage) { super(storage); this.storage = storage; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextPartitionKey.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextPartitionKey.java index 9f0ae25748133..45d20cb012f21 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextPartitionKey.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextPartitionKey.java @@ -28,22 +28,22 @@ public class DLearnContextPartitionKey implements Serializable { /** */ private static final long serialVersionUID = 9005844909381326835L; - /** - * Index of partition. - */ + /** Index of partition. */ private final int part; - /** - * Id of learning context. - */ + /** Id of learning context. */ private final UUID learningCtxId; - /** - * Key of the object. - */ + /** Key of the object. */ private final String key; - /** */ + /** + * Constructs a new instance of learning context partition key. + * + * @param part partition index + * @param learningCtxId learning context id + * @param key key + */ public DLearnContextPartitionKey(int part, UUID learningCtxId, String key) { this.part = part; this.learningCtxId = learningCtxId; From 7ebd20a6ef1eca6063013a5f8a8361ae4f76a609 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Wed, 24 Jan 2018 01:23:24 +0300 Subject: [PATCH 14/38] IGNITE-7437 Introduce IDD (Ignite Distributed Dataset) as a prototype. --- .../java/org/apache/ignite/ml/idd/IDD.java | 8 ++ .../org/apache/ignite/ml/idd/IDDFactory.java | 68 +++++++++++++++++ .../org/apache/ignite/ml/idd/IDDImpl.java | 69 +++++++++++++++++ .../ml/idd/IDDInstanceLocalStorage.java | 20 +++++ .../apache/ignite/ml/idd/IDDPartition.java | 21 ++++++ .../idd/IDDPartitionDistributedSegment.java | 40 ++++++++++ .../apache/ignite/ml/idd/IDDPartitionKey.java | 23 ++++++ .../org/apache/ignite/ml/IDDPlayground.java | 74 +++++++++++++++++++ 8 files changed, 323 insertions(+) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/idd/IDD.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDFactory.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDImpl.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDInstanceLocalStorage.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartition.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartitionDistributedSegment.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartitionKey.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/IDDPlayground.java diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDD.java b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDD.java new file mode 100644 index 0000000000000..dac76d01bc767 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDD.java @@ -0,0 +1,8 @@ +package org.apache.ignite.ml.idd; + +import org.apache.ignite.ml.math.functions.IgniteConsumer; + +public interface IDD { + + public void compute(IgniteConsumer> consumer); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDFactory.java new file mode 100644 index 0000000000000..ce2d5d54e6510 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDFactory.java @@ -0,0 +1,68 @@ +package org.apache.ignite.ml.idd; + +import java.util.List; +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cache.affinity.AffinityFunction; +import org.apache.ignite.cache.affinity.AffinityFunctionContext; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +public class IDDFactory { + + @SuppressWarnings("unchecked") + public static IDD createIDD(Ignite ignite, IgniteCache upstreamCache, + IgniteBiFunction, Integer, L> loader) { + + UUID iddId = UUID.randomUUID(); + + CacheConfiguration> cc = new CacheConfiguration<>(); + cc.setName(iddId.toString()); + AffinityFunction af = upstreamCache.getConfiguration(CacheConfiguration.class).getAffinity(); + cc.setAffinity(new AffinityFunctionWrapper(af)); + IgniteCache> iddCache = ignite.createCache(cc); + + Affinity affinity = ignite.affinity(iddCache.getName()); + int partitions = affinity.partitions(); + + for (int part = 0; part < partitions; part++) { + IDDPartitionDistributedSegment distributedSegment + = new IDDPartitionDistributedSegment<>(loader, iddId, part, null); + iddCache.put(part, distributedSegment); + } + + return new IDDImpl<>(ignite, upstreamCache, iddCache, iddId); + } + + private static class AffinityFunctionWrapper implements AffinityFunction { + + private final AffinityFunction delegate; + + public AffinityFunctionWrapper(AffinityFunction delegate) { + this.delegate = delegate; + } + + @Override public void reset() { + delegate.reset(); + } + + @Override public int partitions() { + return delegate.partitions(); + } + + @Override public int partition(Object key) { + return (Integer) key; + } + + @Override public List> assignPartitions(AffinityFunctionContext affCtx) { + return delegate.assignPartitions(affCtx); + } + + @Override public void removeNode(UUID nodeId) { + delegate.removeNode(nodeId); + } + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDImpl.java b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDImpl.java new file mode 100644 index 0000000000000..bbe1e81b6fab9 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDImpl.java @@ -0,0 +1,69 @@ +package org.apache.ignite.ml.idd; + +import java.util.Arrays; +import java.util.UUID; +import java.util.concurrent.ConcurrentMap; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteConsumer; + +public class IDDImpl implements IDD { + + private final Ignite ignite; + + private final IgniteCache upstreamCache; + + private final IgniteCache> iddCache; + + private final UUID iddId; + + public IDDImpl(Ignite ignite, IgniteCache upstreamCache, + IgniteCache> iddCache, UUID iddId) { + this.ignite = ignite; + this.upstreamCache = upstreamCache; + this.iddCache = iddCache; + this.iddId = iddId; + } + + @SuppressWarnings("unchecked") + @Override public void compute(IgniteConsumer> consumer) { + Affinity affinity = ignite.affinity(iddCache.getName()); + int partitions = affinity.partitions(); + + String upstreamCacheName = upstreamCache.getName(); + String iddCacheName = iddCache.getName(); + + for (int part = 0; part < partitions; part++) { + int currPart = part; + ignite.compute().affinityRun( + Arrays.asList(iddCacheName, upstreamCacheName), + currPart, + () -> { + Ignite localIgnite = Ignition.localIgnite(); + + IgniteCache> localIddCache = localIgnite.cache(iddCacheName); + IDDPartitionDistributedSegment distributedSegment = localIddCache.get(currPart); + + IDDInstanceLocalStorage localStorage = IDDInstanceLocalStorage.getInstance(); + ConcurrentMap localPartStorage = localStorage.getOrCreateIDDLocalStorage(iddId); + + L localSegment = (L) localPartStorage.computeIfAbsent(currPart, p -> { + IgniteCache localUpstreamCache = ignite.cache(upstreamCacheName); + IgniteBiFunction, Integer, L> loader = distributedSegment.getLoader(); + return loader.apply(localUpstreamCache, currPart); + }); + + if (localSegment == null) + throw new IllegalStateException(); + + IDDPartition partition = new IDDPartition<>(distributedSegment, localSegment); + + consumer.accept(partition); + } + ); + } + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDInstanceLocalStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDInstanceLocalStorage.java new file mode 100644 index 0000000000000..e6b23c975eb23 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDInstanceLocalStorage.java @@ -0,0 +1,20 @@ +package org.apache.ignite.ml.idd; + +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +public class IDDInstanceLocalStorage { + + private static final IDDInstanceLocalStorage instance = new IDDInstanceLocalStorage(); + + private final ConcurrentMap> storage = new ConcurrentHashMap<>(); + + public static IDDInstanceLocalStorage getInstance() { + return instance; + } + + public ConcurrentMap getOrCreateIDDLocalStorage(UUID iddId) { + return storage.computeIfAbsent(iddId, i -> new ConcurrentHashMap<>()); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartition.java new file mode 100644 index 0000000000000..23ddbc1dc1696 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartition.java @@ -0,0 +1,21 @@ +package org.apache.ignite.ml.idd; + +public class IDDPartition { + + private final IDDPartitionDistributedSegment distributedSegment; + + private final L localSegment; + + public IDDPartition(IDDPartitionDistributedSegment distributedSegment, L localSegment) { + this.distributedSegment = distributedSegment; + this.localSegment = localSegment; + } + + public IDDPartitionDistributedSegment getDistributedSegment() { + return distributedSegment; + } + + public L getLocalSegment() { + return localSegment; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartitionDistributedSegment.java b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartitionDistributedSegment.java new file mode 100644 index 0000000000000..febe6a4b4caeb --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartitionDistributedSegment.java @@ -0,0 +1,40 @@ +package org.apache.ignite.ml.idd; + +import java.util.UUID; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +public class IDDPartitionDistributedSegment { + + private final IgniteBiFunction, Integer, L> loader; + + private final UUID iddId; + + private final int partId; + + private final D distributedSegment; + + public IDDPartitionDistributedSegment( + IgniteBiFunction, Integer, L> loader, UUID iddId, int partId, D distributedSegment) { + this.loader = loader; + this.iddId = iddId; + this.partId = partId; + this.distributedSegment = distributedSegment; + } + + public IgniteBiFunction, Integer, L> getLoader() { + return loader; + } + + public UUID getIddId() { + return iddId; + } + + public int getPartId() { + return partId; + } + + public D getDistributedSegment() { + return distributedSegment; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartitionKey.java b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartitionKey.java new file mode 100644 index 0000000000000..944fcd8a55f74 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartitionKey.java @@ -0,0 +1,23 @@ +package org.apache.ignite.ml.idd; + +import java.util.UUID; + +public class IDDPartitionKey { + + private final UUID iddId; + + private final int partId; + + public IDDPartitionKey(UUID iddId, int partId) { + this.iddId = iddId; + this.partId = partId; + } + + public UUID getIddId() { + return iddId; + } + + public int getPartId() { + return partId; + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/IDDPlayground.java b/modules/ml/src/test/java/org/apache/ignite/ml/IDDPlayground.java new file mode 100644 index 0000000000000..994b32706e94f --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/IDDPlayground.java @@ -0,0 +1,74 @@ +package org.apache.ignite.ml; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.ml.idd.IDD; +import org.apache.ignite.ml.idd.IDDFactory; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +public class IDDPlayground extends GridCommonAbstractTest { + /** Number of nodes in grid */ + private static final int NODE_COUNT = 4; + + /** */ + private Ignite ignite; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + for (int i = 1; i <= NODE_COUNT; i++) + startGrid(i); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() { + stopAllGrids(); + } + + /** + * {@inheritDoc} + */ + @Override protected void beforeTest() throws Exception { + /* Grid instance. */ + ignite = grid(NODE_COUNT); + ignite.configuration().setPeerClassLoadingEnabled(true); + IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); + } + + public void testTest() { + CacheConfiguration cc = new CacheConfiguration<>(); + cc.setAffinity(new RendezvousAffinityFunction(false, 10)); + cc.setName("TEST"); + IgniteCache cache = ignite.createCache(cc); + for (int i = 0; i < 100; i++) { + cache.put(i, i); + } + + IDD idd = IDDFactory.createIDD(ignite, cache, (c, p) -> { + System.err.println("LOAD " + p); + ScanQuery q = new ScanQuery<>(); + q.setLocal(true); + q.setPartition(p); + List values = new ArrayList<>(); + c.query(q).forEach(e -> values.add(e.getValue())); + int[] res = new int[values.size()]; + for (int i = 0; i < res.length; i++) { + res[i] = values.get(i); + } + return res; + }); + + for (int i = 0; i < 3; i++) { + idd.compute(part -> { + System.err.println("PART " + part.getDistributedSegment().getPartId() + " : " + + Arrays.toString(part.getLocalSegment())); + }); + } + } +} From 3c11f600b920fe19f57ca4203be5ac44219c4f52 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Wed, 24 Jan 2018 19:50:12 +0300 Subject: [PATCH 15/38] IGNITE-7437 Introduce DLC (Distributed Learning Context). --- .../java/org/apache/ignite/ml/dlc/DLC.java | 122 ++++++++++++ .../org/apache/ignite/ml/dlc/DLCFactory.java | 95 +++++++++ .../apache/ignite/ml/dlc/DLCPartition.java | 77 ++++++++ ...LCPartitionRecoverableDataTransformer.java | 37 ++++ ...DLCPartitionReplicatedDataTransformer.java | 36 ++++ .../ml/dlc/DLCPartitionTransformer.java | 58 ++++++ .../ignite/ml/dlc/dataset/DLCDataset.java | 41 ++++ .../ml/dlc/dataset/DLCLabeledDataset.java | 42 ++++ .../ignite/ml/dlc/dataset/DLCWrapper.java | 58 ++++++ .../DLCDatasetPartitionRecoverable.java | 35 ++++ ...DLCLabeledDatasetPartitionRecoverable.java | 32 +++ .../dataset/transformer/DLCTransformers.java | 39 ++++ .../UpstreamToDatasetTransformer.java | 62 ++++++ .../UpstreamToLabeledDatasetTransformer.java | 74 +++++++ .../ml/dlc/impl/cache/CacheBasedDLCImpl.java | 158 +++++++++++++++ .../cache/DLCAffinityFunctionWrapper.java | 75 ++++++++ .../util/CacheBasedDLCPartitionBuilder.java | 182 ++++++++++++++++++ .../DLCPartitionRecoverableDataStorage.java | 46 +++++ .../UpstreamPartitionNotFoundException.java | 46 +++++ .../ml/dlc/impl/local/MapBasedDLCImpl.java | 117 +++++++++++ .../java/org/apache/ignite/ml/idd/IDD.java | 8 - .../org/apache/ignite/ml/idd/IDDFactory.java | 68 ------- .../org/apache/ignite/ml/idd/IDDImpl.java | 69 ------- .../ml/idd/IDDInstanceLocalStorage.java | 20 -- .../apache/ignite/ml/idd/IDDPartition.java | 21 -- .../idd/IDDPartitionDistributedSegment.java | 40 ---- .../apache/ignite/ml/idd/IDDPartitionKey.java | 23 --- ...{IDDPlayground.java => DLCPlayground.java} | 45 ++--- 28 files changed, 1453 insertions(+), 273 deletions(-) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLC.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCFactory.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartition.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableDataTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedDataTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCDataset.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCLabeledDataset.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCWrapper.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCDatasetPartitionRecoverable.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCLabeledDatasetPartitionRecoverable.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/DLCTransformers.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToDatasetTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToLabeledDatasetTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/CacheBasedDLCImpl.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/DLCAffinityFunctionWrapper.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/CacheBasedDLCPartitionBuilder.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCPartitionRecoverableDataStorage.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/UpstreamPartitionNotFoundException.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCImpl.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/idd/IDD.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDFactory.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDImpl.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDInstanceLocalStorage.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartition.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartitionDistributedSegment.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartitionKey.java rename modules/ml/src/test/java/org/apache/ignite/ml/{IDDPlayground.java => DLCPlayground.java} (57%) diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLC.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLC.java new file mode 100644 index 0000000000000..c6a4d20bc6cc3 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLC.java @@ -0,0 +1,122 @@ +/* + * 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.ml.dlc; + +import java.io.Serializable; +import org.apache.ignite.ml.math.functions.IgniteBiConsumer; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; +import org.apache.ignite.ml.math.functions.IgniteConsumer; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +/** + * Distributed Learning Context provides the API which allows to perform iterative computation tasks on a distributed + * datasets. Every computation performed via Distributed Learning Context works with {@link DLCPartition} which consists + * of replicated data and recoverable data. Computation task can modify these segments to maintain the iterative + * algorithm context. + * + * @param type of an upstream value key + * @param type of an upstream value + * @param type of replicated data of a partition + * @param type of recoverable data of a partition + */ +public interface DLC extends AutoCloseable { + /** + * Computes a given function on every DLC partition in current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param mapper mapper function applied on every partition + * @param reducer reducer of the results + * @param identity identity value + * @param result type + * @return final reduced result + */ + public R compute(IgniteBiFunction, Integer, R> mapper, IgniteBinaryOperator reducer, + R identity); + + /** + * Computes a given function on every DLC partition in current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param mapper mapper function applied on every partition + * @param reducer reducer of the results + * @param result type + * @return final reduced result + */ + default public R compute(IgniteBiFunction, Integer, R> mapper, + IgniteBinaryOperator reducer) { + return compute(mapper, reducer, null); + } + + /** + * Computes a given function on every DLC partition in current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param mapper mapper function applied on every partition + * @param reducer reducer of the results + * @param identity identity value + * @param result type + * @return final reduced result + */ + default public R compute(IgniteFunction, R> mapper, IgniteBinaryOperator reducer, + R identity) { + return compute((part, partIdx) -> mapper.apply(part), reducer, identity); + } + + /** + * Computes a given function on every DLC partition in current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param mapper mapper function applied on every partition + * @param reducer reducer of the results + * @param result type + * @return final reduced result + */ + default public R compute(IgniteFunction, R> mapper, IgniteBinaryOperator reducer) { + return compute((part, partIdx) -> mapper.apply(part), reducer); + } + + /** + * Computes a given function on every DLC partition in current learning context independently. The goal of this + * approach is to perform {@code mapper} locally on the nodes where partitions are placed and do not involve network + * subsystem where it's possible. + * + * @param mapper mapper function applied on every partition + */ + default public void compute(IgniteBiConsumer, Integer> mapper) { + compute((part, partIdx) -> { + mapper.accept(part, partIdx); + return null; + }, (a, b) -> null); + } + + /** + * Computes a given function on every DLC partition in current learning context independently. The goal of this + * approach is to perform {@code mapper} locally on the nodes where partitions are placed and do not involve network + * subsystem where it's possible. + * + * @param mapper mapper function applied on every partition + */ + default public void compute(IgniteConsumer> mapper) { + compute((part, partIdx) -> mapper.accept(part)); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCFactory.java new file mode 100644 index 0000000000000..e9a8fdf7b6403 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCFactory.java @@ -0,0 +1,95 @@ +/* + * 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.ml.dlc; + +import java.io.Serializable; +import java.util.Collections; +import java.util.UUID; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cache.affinity.AffinityFunction; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.ml.dlc.impl.cache.CacheBasedDLCImpl; +import org.apache.ignite.ml.dlc.impl.cache.DLCAffinityFunctionWrapper; + +/** + * Distributed Learning Context factory which produces contexts based on Ignite Cache and local Map. + */ +public class DLCFactory { + /** + * Constructs a new instance of Distributed Learning Context based on the specified upstream Ignite Cache and uses + * Ignite Cache as reliable storage. + * + * @param ignite ignite instance + * @param upstreamCache upstream cache + * @param replicatedDataLdr replicated data loader + * @param recoverableDataLdr recovered data loader + * @param type of an upstream value key + * @param type of an upstream value + * @param type of replicated data of a partition + * @param type of recoverable data of a partition + * @return Distributed Learning Context + */ + @SuppressWarnings("unchecked") + public static DLC createIDD(Ignite ignite, + IgniteCache upstreamCache, + DLCPartitionReplicatedDataTransformer replicatedDataLdr, + DLCPartitionRecoverableDataTransformer recoverableDataLdr) { + + UUID dlcId = UUID.randomUUID(); + + AffinityFunction upstreamCacheAffinity = upstreamCache.getConfiguration(CacheConfiguration.class).getAffinity(); + + CacheConfiguration> dlcCacheCfg = new CacheConfiguration<>(); + dlcCacheCfg.setName(dlcId.toString()); + dlcCacheCfg.setAffinity(new DLCAffinityFunctionWrapper(upstreamCacheAffinity)); + + IgniteCache> dlcCache = ignite.createCache(dlcCacheCfg); + + Affinity affinity = ignite.affinity(dlcCache.getName()); + int partitions = affinity.partitions(); + + for (int partIdx = 0; partIdx < partitions; partIdx++) { + int currPartIdx = partIdx; + + ignite.compute().affinityRun(Collections.singletonList(dlcCache.getName()), partIdx, () -> { + Ignite locIgnite = Ignition.localIgnite(); + IgniteCache locUpstreamCache = locIgnite.cache(upstreamCache.getName()); + + ScanQuery qry = new ScanQuery<>(); + qry.setLocal(true); + qry.setPartition(currPartIdx); + + long cnt = locUpstreamCache.localSizeLong(currPartIdx); + Q replicatedData; + try (QueryCursor> cursor = locUpstreamCache.query(qry)) { + replicatedData = replicatedDataLdr.apply(cursor, cnt); + } + DLCPartition part = new DLCPartition<>(replicatedData, recoverableDataLdr); + dlcCache.put(currPartIdx, part); + }); + } + + return new CacheBasedDLCImpl<>(ignite, upstreamCache, dlcCache, dlcId); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartition.java new file mode 100644 index 0000000000000..6ddc75d810dfd --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartition.java @@ -0,0 +1,77 @@ +/* + * 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.ml.dlc; + +import java.io.Serializable; + +/** + * Distributed Learning Context partition which consists of replicated data and recoverable data. Replicated + * data is stored in a reliable storage (Ignite Cache) and in case of the node failure or rebalancing automatically + * restored on another node. Recoverable data is stored in a non-reliable local storage and in case of node failure or + * rebalancing when partition is restored on another node should be reloaded from the upstream. + * + * @param type of an upstream value key + * @param type of an upstream value + * @param type of replicated data of a partition + * @param type of recoverable data of a partition + */ +public class DLCPartition implements Serializable { + /** */ + private static final long serialVersionUID = -6348461866724022880L; + + /** Replicated data. */ + private final Q replicatedData; + + /** Loader of the recoverable part of this partition. */ + private final DLCPartitionRecoverableDataTransformer recoverableDataTransformer; + + /** Recoverable data. */ + private transient W recoverableData; + + /** + * Constructs a new instance of a DLC partition. + * + * @param replicatedData replicated data + * @param recoverableDataTransformer transformer of the recoverable part of this partition + */ + public DLCPartition(Q replicatedData, + DLCPartitionRecoverableDataTransformer recoverableDataTransformer) { + this.replicatedData = replicatedData; + this.recoverableDataTransformer = recoverableDataTransformer; + } + + /** */ + public Q getReplicatedData() { + return replicatedData; + } + + /** */ + public DLCPartitionRecoverableDataTransformer getRecoverableDataTransformer() { + return recoverableDataTransformer; + } + + /** */ + public W getRecoverableData() { + return recoverableData; + } + + /** */ + public void setRecoverableData(W recoverableData) { + this.recoverableData = recoverableData; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableDataTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableDataTransformer.java new file mode 100644 index 0000000000000..f7a7ac1696b28 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableDataTransformer.java @@ -0,0 +1,37 @@ +/* + * 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.ml.dlc; + +import java.io.Serializable; +import javax.cache.Cache; +import org.apache.ignite.ml.math.functions.IgniteTriFunction; + +/** + * Transformer which transforms upstream data to any desired type of recoverable data. + * + * @param type of an upstream value key + * @param type of an upstream value + * @param type of replicated data of a partition + * @param type of recoverable data of a partition + */ +@FunctionalInterface +public interface DLCPartitionRecoverableDataTransformer + extends IgniteTriFunction>, Long, Q, W> { + /** {@inheritDoc} */ + @Override public W apply(Iterable> upstreamData, Long upstreamDataSize, Q replicatedData); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedDataTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedDataTransformer.java new file mode 100644 index 0000000000000..9bf0d18df26fb --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedDataTransformer.java @@ -0,0 +1,36 @@ +/* + * 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.ml.dlc; + +import java.io.Serializable; +import javax.cache.Cache; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +/** + * Transformer which transforms upstream data to any desired type of replicated data. + * + * @param type of an upstream value key + * @param type of an upstream value + * @param type of replicated data + */ +@FunctionalInterface +public interface DLCPartitionReplicatedDataTransformer + extends IgniteBiFunction>, Long, Q> { + /** {@inheritDoc} */ + @Override public Q apply(Iterable> upstreamData, Long upstreamDataSize); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionTransformer.java new file mode 100644 index 0000000000000..42165d330614c --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionTransformer.java @@ -0,0 +1,58 @@ +/* + * 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.ml.dlc; + +import java.io.Serializable; +import javax.cache.Cache; + +/** + * Distributed Learning Dataset partition transformer. + * + * @param type of an upstream value key + * @param type of an upstream value + * @param type of replicated data of a partition + * @param type of recoverable data of a partition + */ +public interface DLCPartitionTransformer> extends Serializable { + /** + * Transformer which transforms upstream data to any desired type of recoverable data. + * + * @param upstreamData upstream data + * @param upstreamDataSize upstream data size + * @param replicatedData replicated data + * @return recoverable data + */ + public W transformRecoverablePart(Iterable> upstreamData, Long upstreamDataSize, Q replicatedData); + + /** + * Transformer which transforms upstream data to any desired type of replicated data. + * + * @param upstreamData upstream data + * @param upstreamDataSize upstream data size + * @return replicated data + */ + public Q transformReplicatedPart(Iterable> upstreamData, Long upstreamDataSize); + + /** + * Wraps DLC to provide partition-specific API. + * + * @param ctx distributed learning context + * @return wrapped context + */ + public I wrapDLC(DLC ctx); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCDataset.java new file mode 100644 index 0000000000000..952b754dbbfad --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCDataset.java @@ -0,0 +1,41 @@ +/* + * 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.ml.dlc.dataset; + +import java.io.Serializable; +import org.apache.ignite.ml.dlc.DLC; +import org.apache.ignite.ml.dlc.dataset.part.recoverable.DLCDatasetPartitionRecoverable; + +/** + * Dataset provides API to work with dataset. + * + * @param type of an upstream value key + * @param type of an upstream value + * @param type of replicated data of a partition + */ +public class DLCDataset extends DLCWrapper { + /** + * Constructs a new instance of Distributed Learning Context wrapper + * + * @param delegate delegate which actually performs base functions + */ + public DLCDataset( + DLC delegate) { + super(delegate); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCLabeledDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCLabeledDataset.java new file mode 100644 index 0000000000000..f64151c83222b --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCLabeledDataset.java @@ -0,0 +1,42 @@ +/* + * 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.ml.dlc.dataset; + +import java.io.Serializable; +import org.apache.ignite.ml.dlc.DLC; +import org.apache.ignite.ml.dlc.dataset.part.recoverable.DLCLabeledDatasetPartitionRecoverable; + +/** + * Dataset provides API to work with labeled dataset. + * + * @param type of an upstream value key + * @param type of an upstream value + * @param type of replicated data of a partition + */ +public class DLCLabeledDataset + extends DLCWrapper { + /** + * Constructs a new instance of Distributed Learning Context wrapper + * + * @param delegate delegate which actually performs base functions + */ + public DLCLabeledDataset( + DLC delegate) { + super(delegate); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCWrapper.java new file mode 100644 index 0000000000000..f04470eacc7e5 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCWrapper.java @@ -0,0 +1,58 @@ +/* + * 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.ml.dlc.dataset; + +import java.io.Serializable; +import org.apache.ignite.ml.dlc.DLC; +import org.apache.ignite.ml.dlc.DLCPartition; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; + +/** + * Wrapper of a Distributed Learning Context which allows to introduce new context-specific methods based on base + * {@code compute()} functionality. + * + * @param type of an upstream value key + * @param type of an upstream value + * @param type of replicated data of a partition + * @param type of recoverable data of a partition + */ +public class DLCWrapper implements DLC { + /** Delegate which actually performs base functions like {@code compute()} and {@code close()}. */ + private final DLC delegate; + + /** + * Constructs a new instance of Distributed Learning Context wrapper + * + * @param delegate delegate which actually performs base functions + */ + public DLCWrapper(DLC delegate) { + this.delegate = delegate; + } + + /** {@inheritDoc} */ + @Override public R compute(IgniteBiFunction, Integer, R> mapper, IgniteBinaryOperator reducer, + R identity) { + return delegate.compute(mapper, reducer, identity); + } + + /** {@inheritDoc} */ + @Override public void close() throws Exception { + delegate.close(); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCDatasetPartitionRecoverable.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCDatasetPartitionRecoverable.java new file mode 100644 index 0000000000000..9441f11793371 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCDatasetPartitionRecoverable.java @@ -0,0 +1,35 @@ +/* + * 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.ml.dlc.dataset.part.recoverable; + +public class DLCDatasetPartitionRecoverable implements AutoCloseable { + + private final double[][] data; + + public DLCDatasetPartitionRecoverable(double[][] data) { + this.data = data; + } + + public double[][] getData() { + return data; + } + + @Override public void close() { + + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCLabeledDatasetPartitionRecoverable.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCLabeledDatasetPartitionRecoverable.java new file mode 100644 index 0000000000000..ebb9c62bd3ef4 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCLabeledDatasetPartitionRecoverable.java @@ -0,0 +1,32 @@ +/* + * 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.ml.dlc.dataset.part.recoverable; + +public class DLCLabeledDatasetPartitionRecoverable extends DLCDatasetPartitionRecoverable { + + private double[] labels; + + public DLCLabeledDatasetPartitionRecoverable(double[][] data, double[] labels) { + super(data); + this.labels = labels; + } + + public double[] getLabels() { + return labels; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/DLCTransformers.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/DLCTransformers.java new file mode 100644 index 0000000000000..a2970cfb1f208 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/DLCTransformers.java @@ -0,0 +1,39 @@ +/* + * 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.ml.dlc.dataset.transformer; + +import javax.cache.Cache; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +/** + * Aggregator which allows to find desired transformer from one partition data type to another. This class doesn't + * introduce a new functionality, but helps to work efficiently with existing transformers. + */ +public class DLCTransformers { + + public static UpstreamToDatasetTransformer upstreamToDataset( + IgniteFunction, double[]> featureExtractor, int features) { + return new UpstreamToDatasetTransformer<>(featureExtractor, features); + } + + public static UpstreamToLabeledDatasetTransformer upstreamToLabeledDataset( + IgniteFunction, double[]> featureExtractor, + IgniteFunction, Double> lbExtractor, int features) { + return new UpstreamToLabeledDatasetTransformer<>(featureExtractor, lbExtractor, features); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToDatasetTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToDatasetTransformer.java new file mode 100644 index 0000000000000..c48c51a42fd8f --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToDatasetTransformer.java @@ -0,0 +1,62 @@ +/* + * 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.ml.dlc.dataset.transformer; + +import java.io.Serializable; +import javax.cache.Cache; +import org.apache.ignite.ml.dlc.DLCPartitionRecoverableDataTransformer; +import org.apache.ignite.ml.dlc.dataset.part.recoverable.DLCDatasetPartitionRecoverable; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +/** + * Transforms upstream data into dataset recoverable part of partition using specified feature extractor. + * + * @param type of an upstream value key + * @param type of an upstream value + */ +public class UpstreamToDatasetTransformer implements DLCPartitionRecoverableDataTransformer { + /** */ + private static final long serialVersionUID = -3713681392540367983L; + + /** Feature extractor. */ + private final IgniteFunction, double[]> featureExtractor; + + /** Number of features. */ + private final int features; + + /** + * Constructs a new instance of transformer. + * + * @param featureExtractor feature extractor + * @param features number of features + */ + protected UpstreamToDatasetTransformer( + IgniteFunction, double[]> featureExtractor, int features) { + this.featureExtractor = featureExtractor; + this.features = features; + } + + /** {@inheritDoc} */ + @Override public DLCDatasetPartitionRecoverable apply(Iterable> entries, Long aLong, Serializable serializable) { + double[][] features = new double[Math.toIntExact(aLong)][]; + int ptr = 0; + for (Cache.Entry e : entries) + features[ptr++] = featureExtractor.apply(e); + return new DLCDatasetPartitionRecoverable(features); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToLabeledDatasetTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToLabeledDatasetTransformer.java new file mode 100644 index 0000000000000..f4ebe50128fa0 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToLabeledDatasetTransformer.java @@ -0,0 +1,74 @@ +/* + * 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.ml.dlc.dataset.transformer; + +import java.io.Serializable; +import javax.cache.Cache; +import org.apache.ignite.ml.dlc.DLCPartitionRecoverableDataTransformer; +import org.apache.ignite.ml.dlc.dataset.part.recoverable.DLCLabeledDatasetPartitionRecoverable; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +/** + * Transforms upstream data into labeled dataset recoverable part of partition using specified feature extractor and + * label extractor. + * + * @param type of an upstream value key + * @param type of an upstream value + */ +public class UpstreamToLabeledDatasetTransformer implements DLCPartitionRecoverableDataTransformer { + /** */ + private static final long serialVersionUID = -1224768715207401297L; + + /** Feature extractor. */ + private final IgniteFunction, double[]> featureExtractor; + + /** Label extractor. */ + private final IgniteFunction, Double> lbExtractor; + + /** Number of features. */ + private final int features; + + /** + * Constructs a new instance of transformer. + * + * @param featureExtractor feature extractor + * @param lbExtractor label extractor + * @param features number of features + */ + protected UpstreamToLabeledDatasetTransformer( + IgniteFunction, double[]> featureExtractor, + IgniteFunction, Double> lbExtractor, int features) { + this.featureExtractor = featureExtractor; + this.lbExtractor = lbExtractor; + this.features = features; + } + + /** {@inheritDoc} */ + @Override public DLCLabeledDatasetPartitionRecoverable apply(Iterable> entries, Long aLong, + Serializable ser) { + double[][] features = new double[Math.toIntExact(aLong)][]; + double[] labels = new double[Math.toIntExact(aLong)]; + int ptr = 0; + for (Cache.Entry e : entries) { + features[ptr] = featureExtractor.apply(e); + labels[ptr] = lbExtractor.apply(e); + ptr++; + } + return new DLCLabeledDatasetPartitionRecoverable(features, labels); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/CacheBasedDLCImpl.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/CacheBasedDLCImpl.java new file mode 100644 index 0000000000000..d7c9dfc911e77 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/CacheBasedDLCImpl.java @@ -0,0 +1,158 @@ +/* + * 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.ml.dlc.impl.cache; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cluster.ClusterGroup; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.ml.dlc.DLC; +import org.apache.ignite.ml.dlc.DLCPartition; +import org.apache.ignite.ml.dlc.impl.cache.util.CacheBasedDLCPartitionBuilder; +import org.apache.ignite.ml.dlc.impl.cache.util.UpstreamPartitionNotFoundException; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; + +/** + * Cache based implementation of a Distributed Learning Context. + * + * @param type of an upstream value key + * @param type of an upstream value + * @param type of replicated data of a partition + * @param type of recoverable data of a partition + */ +public class CacheBasedDLCImpl implements DLC { + /** Ignite instance. */ + private final Ignite ignite; + + /** Upstream cache. */ + private final IgniteCache upstreamCache; + + /** Distributed Learning Context cache. */ + private final IgniteCache> dlcCache; + + /** Distributed Learning Context id. */ + private final UUID dlcId; + + /** + * Constructs a new instance of a cache based distributed learning context. + * + * @param ignite ignite instance + * @param upstreamCache upstream cache + * @param dlcCache distributed learning context cache + * @param dlcId distributed learning context id + */ + public CacheBasedDLCImpl(Ignite ignite, IgniteCache upstreamCache, + IgniteCache> dlcCache, UUID dlcId) { + this.ignite = ignite; + this.upstreamCache = upstreamCache; + this.dlcCache = dlcCache; + this.dlcId = dlcId; + } + + /** {@inheritDoc} */ + @Override public R compute(IgniteBiFunction, Integer, R> mapper, IgniteBinaryOperator reducer, + R identity) { + Affinity affinity = ignite.affinity(dlcCache.getName()); + int partitions = affinity.partitions(); + + String upstreamCacheName = upstreamCache.getName(); + String iddCacheName = dlcCache.getName(); + + Map> calls = new HashMap<>(); + Map> futures = new HashMap<>(); + + ClusterGroup clusterGrp = ignite.cluster().forDataNodes(iddCacheName); + + for (int partIdx = 0; partIdx < partitions; partIdx++) { + final int currPartIdx = partIdx; + + IgniteCallable call = () -> { + CacheBasedDLCPartitionBuilder partBuilder = new CacheBasedDLCPartitionBuilder<>( + Ignition.localIgnite(), + upstreamCacheName, + iddCacheName, + dlcId, + currPartIdx + ); + + DLCPartition part = partBuilder.build(); + + R partRes = mapper.apply(part, currPartIdx); + + IgniteCache> dlcCache = ignite.cache(iddCacheName); + + dlcCache.put(currPartIdx, part); + + return partRes; + }; + + IgniteFuture fut = ignite.compute(clusterGrp).affinityCallAsync( + Arrays.asList(iddCacheName, upstreamCacheName), + currPartIdx, + call + ); + + calls.put(currPartIdx, call); + futures.put(currPartIdx, fut); + } + + R res = identity; + + while (!calls.isEmpty()) { + Iterator>> callIter = calls.entrySet().iterator(); + while (callIter.hasNext()) { + Map.Entry> callEntry = callIter.next(); + + int currPartIdx = callEntry.getKey(); + IgniteCallable call = callEntry.getValue(); + + IgniteFuture fut = futures.get(callEntry.getKey()); + try { + R partRes = fut.get(); + res = reducer.apply(res, partRes); + callIter.remove(); + } + catch (UpstreamPartitionNotFoundException e) { + IgniteFuture newFut = ignite.compute(clusterGrp).affinityCallAsync( + Arrays.asList(iddCacheName, upstreamCacheName), + currPartIdx, + call + ); + futures.put(currPartIdx, newFut); + } + } + } + + return res; + } + + /** {@inheritDoc} */ + @Override public void close() { + ignite.destroyCache(dlcCache.getName()); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/DLCAffinityFunctionWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/DLCAffinityFunctionWrapper.java new file mode 100644 index 0000000000000..261f1357600df --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/DLCAffinityFunctionWrapper.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.ignite.ml.dlc.impl.cache; + +import java.util.List; +import java.util.UUID; +import org.apache.ignite.cache.affinity.AffinityFunction; +import org.apache.ignite.cache.affinity.AffinityFunctionContext; +import org.apache.ignite.cluster.ClusterNode; + +/** + * Affinity function wrapper which uses key as a partition index and delegates all other functions to specified + * delegate. + */ +public class DLCAffinityFunctionWrapper implements AffinityFunction { + /** */ + private static final long serialVersionUID = -8233787063079973753L; + + /** Delegate which actually performs all methods except {@link #partition(Object)}. */ + private final AffinityFunction delegate; + + /** + * Constructs a new instance of a DLC affinity function wrapper. + * + * @param delegate affinity function which actually performs all methods except {@link #partition(Object)} + */ + public DLCAffinityFunctionWrapper(AffinityFunction delegate) { + this.delegate = delegate; + } + + /** {@inheritDoc} */ + @Override public void reset() { + delegate.reset(); + } + + /** {@inheritDoc} */ + @Override public int partitions() { + return delegate.partitions(); + } + + /** + * Returns key as a partition index. + * + * @param key partition index + * @return partition index + */ + @Override public int partition(Object key) { + return (Integer) key; + } + + /** {@inheritDoc} */ + @Override public List> assignPartitions(AffinityFunctionContext affCtx) { + return delegate.assignPartitions(affCtx); + } + + /** {@inheritDoc} */ + @Override public void removeNode(UUID nodeId) { + delegate.removeNode(nodeId); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/CacheBasedDLCPartitionBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/CacheBasedDLCPartitionBuilder.java new file mode 100644 index 0000000000000..6ea13e7dbb99c --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/CacheBasedDLCPartitionBuilder.java @@ -0,0 +1,182 @@ +/* + * 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.ml.dlc.impl.cache.util; + +import java.io.Serializable; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCluster; +import org.apache.ignite.IgniteCompute; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.lang.IgniteRunnable; +import org.apache.ignite.ml.dlc.DLCPartition; +import org.apache.ignite.ml.dlc.DLCPartitionRecoverableDataTransformer; + +/** + * Distributed Learning Context partition builder which constructs a partition from two parts: replicated data which is + * stored in a reliable Ignite Cache and recoverable data which is stored in a local storage and can be recovered from + * the upstream cache. + * + * @param type of an upstream value key + * @param type of an upstream value + * @param type of replicated data of a partition + * @param type of recoverable data of a partition + */ +public class CacheBasedDLCPartitionBuilder { + /** Key used to store local partition storage in the {@link IgniteCluster#nodeLocalMap()}. */ + private static final String NODE_LOCAL_PART_STORAGE_KEY = "ml_local_part_storage"; + + /** Ignite instance. */ + private final Ignite ignite; + + /** Upstream cache name. */ + private final String upstreamCacheName; + + /** Distributed Learning Context cache name. */ + private final String dlcCacheName; + + /** Distributed Learning Context cache name. */ + private final UUID dlcId; + + /** Partition index. */ + private final int partIdx; + + /** + * Constructs a new instance of a cache based Distributed Learning Context partition builder. + * + * @param ignite ignite instance + * @param upstreamCacheName upstream cache name + * @param dlcCacheName distributed learning context cache name + * @param dlcId distributed learning context id + * @param partIdx partition index + */ + public CacheBasedDLCPartitionBuilder(Ignite ignite, String upstreamCacheName, String dlcCacheName, + UUID dlcId, int partIdx) { + this.ignite = ignite; + this.upstreamCacheName = upstreamCacheName; + this.dlcCacheName = dlcCacheName; + this.dlcId = dlcId; + this.partIdx = partIdx; + } + + /** + * Builds a new instance of DLC partition constructed from the replicated and recoverable part. If it's required to + * load recoverable data from the upstream cache but correspondent upstream cache partition is not presented on the + * node the {@link UpstreamPartitionNotFoundException} will be thrown (with assumption that retry can be used in + * this case). + * + * Be aware that this method works correctly only under the condition that partitions of the DLC cache and the + * upstream cache are not moved during the execution. To guarantee this condition please use + * {@link IgniteCompute#affinityCall(String, Object, IgniteCallable)} and + * {@link IgniteCompute#affinityRun(String, Object, IgniteRunnable)} methods or similar to submit the job. + * + * @return distributed learning context partition + */ + public DLCPartition build() { + IgniteCache> dlcCache = ignite.cache(dlcCacheName); + + // Retrieves partition without recoverable data (this data is not stored in DLC cache). + DLCPartition part = dlcCache.get(partIdx); + + DLCPartitionRecoverableDataStorage storage = getLocalPartitionStorage(); + + W recoverableData = storage.getData().get(partIdx); + + // In case when partition just has been moved to the node and haven't been processed here yet recoverable data + // will be empty and we need to load it from the upstream cache. + if (recoverableData == null) { + + // Locks partition loading procedure to avoid multiple memory allocations for the same partition. + Lock partLock = storage.getLocks().computeIfAbsent(partIdx, key -> new ReentrantLock()); + + try { + partLock.lock(); + + // Loads recoverable data from the upstream cache. + recoverableData = storage.getData().computeIfAbsent( + partIdx, + id -> loadRecoverableData(part.getRecoverableDataTransformer(), part.getReplicatedData()) + ); + } + finally { + partLock.unlock(); + } + } + + part.setRecoverableData(recoverableData); + + return part; + } + + /** + * Loads recoverable data from the upstream cache. + * + * @param recoverableDataTransformer recoverable data transformer + * @param replicatedData replicated data + * @return recoverable data + */ + private W loadRecoverableData(DLCPartitionRecoverableDataTransformer recoverableDataTransformer, + Q replicatedData) { + Affinity upstreamCacheAffinity = ignite.affinity(upstreamCacheName); + ClusterNode partNode = upstreamCacheAffinity.mapPartitionToNode(partIdx); + + ClusterNode locNode = ignite.cluster().localNode(); + + // If there is not required partition of the upstream cache on the node throws exception. + if (!partNode.equals(locNode)) + throw new UpstreamPartitionNotFoundException(upstreamCacheName, locNode.id(), partIdx); + + IgniteCache locUpstreamCache = ignite.cache(upstreamCacheName); + + ScanQuery qry = new ScanQuery<>(); + qry.setLocal(true); + qry.setPartition(partIdx); + + // TODO: how to guarantee that cache size will not be changed between these calls? + long cnt = locUpstreamCache.localSizeLong(partIdx); + try (QueryCursor> cursor = locUpstreamCache.query(qry)) { + return recoverableDataTransformer.apply(cursor, cnt, replicatedData); + } + } + + /** + * Retrieves partition recoverable data storage from the {@link IgniteCluster#nodeLocalMap()}. + * + * @return partition recoverable storage + */ + @SuppressWarnings("unchecked") + private DLCPartitionRecoverableDataStorage getLocalPartitionStorage() { + ConcurrentMap nodeLocMap = ignite.cluster().nodeLocalMap(); + + ConcurrentMap> locPartStorage = + (ConcurrentMap>) + nodeLocMap.computeIfAbsent(NODE_LOCAL_PART_STORAGE_KEY, key -> new ConcurrentHashMap<>()); + + return locPartStorage.computeIfAbsent(dlcId, key -> new DLCPartitionRecoverableDataStorage<>()); + } +} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCPartitionRecoverableDataStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCPartitionRecoverableDataStorage.java new file mode 100644 index 0000000000000..2c50897c9fe49 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCPartitionRecoverableDataStorage.java @@ -0,0 +1,46 @@ +/* + * 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.ml.dlc.impl.cache.util; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.locks.Lock; + +/** + * Partition recoverable data storage is a data structure maintained locally on the node and used to keep recoverable + * part of partitions. + * + * @param type of recoverable data + */ +public class DLCPartitionRecoverableDataStorage { + /** Map with partitions index as a key and recoverable data as a value. */ + private final ConcurrentMap data = new ConcurrentHashMap<>(); + + /** Map with partition index as a key and lock as a value. */ + private final ConcurrentMap locks = new ConcurrentHashMap<>(); + + /** */ + public ConcurrentMap getData() { + return data; + } + + /** */ + public ConcurrentMap getLocks() { + return locks; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/UpstreamPartitionNotFoundException.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/UpstreamPartitionNotFoundException.java new file mode 100644 index 0000000000000..0734f5b12c944 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/UpstreamPartitionNotFoundException.java @@ -0,0 +1,46 @@ +/* + * 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.ml.dlc.impl.cache.util; + +import java.io.Serializable; +import java.util.UUID; +import org.apache.ignite.IgniteException; + +/** + * Exception which is thrown when partition is expected to be on the node but it isn't. Assumed reason is that two + * caches with the same affinity function have all their partitions on the same nodes only in terms of eventual + * consistency. + */ +public class UpstreamPartitionNotFoundException extends IgniteException implements Serializable { + /** */ + private static final long serialVersionUID = -8891869046312827676L; + + /** Exception message template. */ + private static final String MSG_TEMPLATE = "Partition %d of %s expected to be on node %s, but it isn't"; + + /** + * Constructs a new instance of an upstream partition not found exception. + * + * @param cacheName cache name + * @param nodeId node id + * @param partIdx partition index + */ + public UpstreamPartitionNotFoundException(String cacheName, UUID nodeId, int partIdx) { + super(String.format(MSG_TEMPLATE, partIdx, cacheName, nodeId.toString())); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCImpl.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCImpl.java new file mode 100644 index 0000000000000..0781ef812cdf0 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCImpl.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.ml.dlc.impl.local; + +import java.io.Serializable; +import java.util.Iterator; +import java.util.Map; +import javax.cache.Cache; +import org.apache.ignite.ml.dlc.DLC; +import org.apache.ignite.ml.dlc.DLCPartition; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; +import org.jetbrains.annotations.NotNull; + +/** + * Local Map based implementation of a Distributed Learning Context. + * + * @param type of an upstream value key + * @param type of an upstream value + * @param type of replicated data of a partition + * @param type of recoverable data of a partition + */ +public class MapBasedDLCImpl, V, Q extends Serializable, W extends AutoCloseable> + implements DLC { + /** Map containing pairs of partition index and partitions. */ + private final Map> dlcMap; + + /** Number of partitions. */ + private final int partitions; + + /** + * Constructs a new instance of a local Map based Distributed Learning Context. + * + * @param dlcMap distributed learning context map + * @param partitions number of partitions + */ + public MapBasedDLCImpl(Map> dlcMap, int partitions) { + this.dlcMap = dlcMap; + this.partitions = partitions; + } + + /** {@inheritDoc} */ + @Override public R compute(IgniteBiFunction, Integer, R> mapper, IgniteBinaryOperator reducer, + R identity) { + + R res = identity; + + for (int partIdx = 0; partIdx < partitions; partIdx++) { + DLCPartition part = dlcMap.get(partIdx); + + R partRes = mapper.apply(part, partIdx); + + res = reducer.apply(res, partRes); + + dlcMap.put(partIdx, part); + } + + return res; + } + + /** {@inheritDoc} */ + @Override public void close() { + dlcMap.clear(); + } + + private class MapBasedCacheEntry implements Cache.Entry { + + private final K key; + + private final V value; + + public MapBasedCacheEntry(K key, V value) { + this.key = key; + this.value = value; + } + + @Override public K getKey() { + return key; + } + + @Override public V getValue() { + return value; + } + + @Override public T unwrap(Class clazz) { + throw new UnsupportedOperationException(); + } + } + + private class MapBasedIterable implements Iterable> { + + private final Iterator> iterator; + + public MapBasedIterable(Iterator> iterator) { + this.iterator = iterator; + } + + @NotNull @Override public Iterator> iterator() { + return iterator; + } + } +} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDD.java b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDD.java deleted file mode 100644 index dac76d01bc767..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDD.java +++ /dev/null @@ -1,8 +0,0 @@ -package org.apache.ignite.ml.idd; - -import org.apache.ignite.ml.math.functions.IgniteConsumer; - -public interface IDD { - - public void compute(IgniteConsumer> consumer); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDFactory.java deleted file mode 100644 index ce2d5d54e6510..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDFactory.java +++ /dev/null @@ -1,68 +0,0 @@ -package org.apache.ignite.ml.idd; - -import java.util.List; -import java.util.UUID; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.cache.affinity.AffinityFunction; -import org.apache.ignite.cache.affinity.AffinityFunctionContext; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; - -public class IDDFactory { - - @SuppressWarnings("unchecked") - public static IDD createIDD(Ignite ignite, IgniteCache upstreamCache, - IgniteBiFunction, Integer, L> loader) { - - UUID iddId = UUID.randomUUID(); - - CacheConfiguration> cc = new CacheConfiguration<>(); - cc.setName(iddId.toString()); - AffinityFunction af = upstreamCache.getConfiguration(CacheConfiguration.class).getAffinity(); - cc.setAffinity(new AffinityFunctionWrapper(af)); - IgniteCache> iddCache = ignite.createCache(cc); - - Affinity affinity = ignite.affinity(iddCache.getName()); - int partitions = affinity.partitions(); - - for (int part = 0; part < partitions; part++) { - IDDPartitionDistributedSegment distributedSegment - = new IDDPartitionDistributedSegment<>(loader, iddId, part, null); - iddCache.put(part, distributedSegment); - } - - return new IDDImpl<>(ignite, upstreamCache, iddCache, iddId); - } - - private static class AffinityFunctionWrapper implements AffinityFunction { - - private final AffinityFunction delegate; - - public AffinityFunctionWrapper(AffinityFunction delegate) { - this.delegate = delegate; - } - - @Override public void reset() { - delegate.reset(); - } - - @Override public int partitions() { - return delegate.partitions(); - } - - @Override public int partition(Object key) { - return (Integer) key; - } - - @Override public List> assignPartitions(AffinityFunctionContext affCtx) { - return delegate.assignPartitions(affCtx); - } - - @Override public void removeNode(UUID nodeId) { - delegate.removeNode(nodeId); - } - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDImpl.java b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDImpl.java deleted file mode 100644 index bbe1e81b6fab9..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDImpl.java +++ /dev/null @@ -1,69 +0,0 @@ -package org.apache.ignite.ml.idd; - -import java.util.Arrays; -import java.util.UUID; -import java.util.concurrent.ConcurrentMap; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.Ignition; -import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteConsumer; - -public class IDDImpl implements IDD { - - private final Ignite ignite; - - private final IgniteCache upstreamCache; - - private final IgniteCache> iddCache; - - private final UUID iddId; - - public IDDImpl(Ignite ignite, IgniteCache upstreamCache, - IgniteCache> iddCache, UUID iddId) { - this.ignite = ignite; - this.upstreamCache = upstreamCache; - this.iddCache = iddCache; - this.iddId = iddId; - } - - @SuppressWarnings("unchecked") - @Override public void compute(IgniteConsumer> consumer) { - Affinity affinity = ignite.affinity(iddCache.getName()); - int partitions = affinity.partitions(); - - String upstreamCacheName = upstreamCache.getName(); - String iddCacheName = iddCache.getName(); - - for (int part = 0; part < partitions; part++) { - int currPart = part; - ignite.compute().affinityRun( - Arrays.asList(iddCacheName, upstreamCacheName), - currPart, - () -> { - Ignite localIgnite = Ignition.localIgnite(); - - IgniteCache> localIddCache = localIgnite.cache(iddCacheName); - IDDPartitionDistributedSegment distributedSegment = localIddCache.get(currPart); - - IDDInstanceLocalStorage localStorage = IDDInstanceLocalStorage.getInstance(); - ConcurrentMap localPartStorage = localStorage.getOrCreateIDDLocalStorage(iddId); - - L localSegment = (L) localPartStorage.computeIfAbsent(currPart, p -> { - IgniteCache localUpstreamCache = ignite.cache(upstreamCacheName); - IgniteBiFunction, Integer, L> loader = distributedSegment.getLoader(); - return loader.apply(localUpstreamCache, currPart); - }); - - if (localSegment == null) - throw new IllegalStateException(); - - IDDPartition partition = new IDDPartition<>(distributedSegment, localSegment); - - consumer.accept(partition); - } - ); - } - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDInstanceLocalStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDInstanceLocalStorage.java deleted file mode 100644 index e6b23c975eb23..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDInstanceLocalStorage.java +++ /dev/null @@ -1,20 +0,0 @@ -package org.apache.ignite.ml.idd; - -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -public class IDDInstanceLocalStorage { - - private static final IDDInstanceLocalStorage instance = new IDDInstanceLocalStorage(); - - private final ConcurrentMap> storage = new ConcurrentHashMap<>(); - - public static IDDInstanceLocalStorage getInstance() { - return instance; - } - - public ConcurrentMap getOrCreateIDDLocalStorage(UUID iddId) { - return storage.computeIfAbsent(iddId, i -> new ConcurrentHashMap<>()); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartition.java deleted file mode 100644 index 23ddbc1dc1696..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartition.java +++ /dev/null @@ -1,21 +0,0 @@ -package org.apache.ignite.ml.idd; - -public class IDDPartition { - - private final IDDPartitionDistributedSegment distributedSegment; - - private final L localSegment; - - public IDDPartition(IDDPartitionDistributedSegment distributedSegment, L localSegment) { - this.distributedSegment = distributedSegment; - this.localSegment = localSegment; - } - - public IDDPartitionDistributedSegment getDistributedSegment() { - return distributedSegment; - } - - public L getLocalSegment() { - return localSegment; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartitionDistributedSegment.java b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartitionDistributedSegment.java deleted file mode 100644 index febe6a4b4caeb..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartitionDistributedSegment.java +++ /dev/null @@ -1,40 +0,0 @@ -package org.apache.ignite.ml.idd; - -import java.util.UUID; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; - -public class IDDPartitionDistributedSegment { - - private final IgniteBiFunction, Integer, L> loader; - - private final UUID iddId; - - private final int partId; - - private final D distributedSegment; - - public IDDPartitionDistributedSegment( - IgniteBiFunction, Integer, L> loader, UUID iddId, int partId, D distributedSegment) { - this.loader = loader; - this.iddId = iddId; - this.partId = partId; - this.distributedSegment = distributedSegment; - } - - public IgniteBiFunction, Integer, L> getLoader() { - return loader; - } - - public UUID getIddId() { - return iddId; - } - - public int getPartId() { - return partId; - } - - public D getDistributedSegment() { - return distributedSegment; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartitionKey.java b/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartitionKey.java deleted file mode 100644 index 944fcd8a55f74..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/idd/IDDPartitionKey.java +++ /dev/null @@ -1,23 +0,0 @@ -package org.apache.ignite.ml.idd; - -import java.util.UUID; - -public class IDDPartitionKey { - - private final UUID iddId; - - private final int partId; - - public IDDPartitionKey(UUID iddId, int partId) { - this.iddId = iddId; - this.partId = partId; - } - - public UUID getIddId() { - return iddId; - } - - public int getPartId() { - return partId; - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/IDDPlayground.java b/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java similarity index 57% rename from modules/ml/src/test/java/org/apache/ignite/ml/IDDPlayground.java rename to modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java index 994b32706e94f..1aa954bb28dfe 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/IDDPlayground.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java @@ -1,19 +1,20 @@ package org.apache.ignite.ml; -import java.util.ArrayList; +import java.io.Serializable; import java.util.Arrays; -import java.util.List; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; -import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.idd.IDD; -import org.apache.ignite.ml.idd.IDDFactory; +import org.apache.ignite.ml.dlc.DLC; +import org.apache.ignite.ml.dlc.DLCFactory; +import org.apache.ignite.ml.dlc.dataset.part.recoverable.DLCDatasetPartitionRecoverable; +import org.apache.ignite.ml.dlc.dataset.transformer.DLCTransformers; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -public class IDDPlayground extends GridCommonAbstractTest { +/** */ +public class DLCPlayground extends GridCommonAbstractTest { /** Number of nodes in grid */ private static final int NODE_COUNT = 4; @@ -41,33 +42,29 @@ public class IDDPlayground extends GridCommonAbstractTest { IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); } + /** */ public void testTest() { CacheConfiguration cc = new CacheConfiguration<>(); - cc.setAffinity(new RendezvousAffinityFunction(false, 10)); + cc.setAffinity(new RendezvousAffinityFunction(false, 2)); cc.setName("TEST"); IgniteCache cache = ignite.createCache(cc); - for (int i = 0; i < 100; i++) { + for (int i = 0; i < 40; i++) cache.put(i, i); - } - IDD idd = IDDFactory.createIDD(ignite, cache, (c, p) -> { - System.err.println("LOAD " + p); - ScanQuery q = new ScanQuery<>(); - q.setLocal(true); - q.setPartition(p); - List values = new ArrayList<>(); - c.query(q).forEach(e -> values.add(e.getValue())); - int[] res = new int[values.size()]; - for (int i = 0; i < res.length; i++) { - res[i] = values.get(i); - } - return res; - }); + DLC idd = DLCFactory.createIDD( + ignite, + cache, + (iter, cnt) -> null, + DLCTransformers.upstreamToDataset(e -> new double[]{1, 2, 3}, 3) + ); for (int i = 0; i < 3; i++) { idd.compute(part -> { - System.err.println("PART " + part.getDistributedSegment().getPartId() + " : " - + Arrays.toString(part.getLocalSegment())); + StringBuilder builder = new StringBuilder(); + builder.append("PART \n"); + for (double[] row : part.getRecoverableData().getData()) + builder.append(Arrays.toString(row)).append("\n"); + System.err.println(builder.toString()); }); } } From 8381bd827cae78478532bf0e55446ff38b1d9e3a Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Thu, 25 Jan 2018 13:35:26 +0300 Subject: [PATCH 16/38] IGNITE-7437 Update DLC and add examples. --- .../ml/dlc/CacheDLCDatasetExample.java | 111 ++++++++++++ .../ml/dlc/LocalDLCDatasetExample.java | 112 ++++++++++++ .../ignite/examples/ml/dlc/package-info.java | 22 +++ .../org/apache/ignite/ml/dlc/DLCFactory.java | 85 +++++++-- .../apache/ignite/ml/dlc/DLCPartition.java | 7 +- ...> DLCPartitionRecoverableTransformer.java} | 26 +-- .../DLCPartitionReplicatedTransformer.java | 40 +++++ .../ml/dlc/DLCPartitionTransformer.java | 58 ------- ...Transformer.java => DLCUpstreamEntry.java} | 39 +++-- .../ignite/ml/dlc/dataset/DLCDataset.java | 162 +++++++++++++++++- .../ml/dlc/dataset/DLCLabeledDataset.java | 9 +- .../ignite/ml/dlc/dataset/DLCWrapper.java | 2 +- .../DLCDatasetPartitionRecoverable.java | 42 ++++- ...DLCLabeledDatasetPartitionRecoverable.java | 55 +++++- .../DLCDatasetPartitionReplicated.java | 28 +++ .../DLCLabeledDatasetPartitionReplicated.java | 28 +++ .../dataset/transformer/DLCTransformers.java | 35 +++- .../UpstreamToDatasetTransformer.java | 62 ------- .../UpstreamToLabeledDatasetTransformer.java | 74 -------- .../UpstreamToDatasetTransformer.java | 84 +++++++++ .../UpstreamToLabeledDatasetTransformer.java | 94 ++++++++++ .../util/CacheBasedDLCPartitionBuilder.java | 8 +- .../cache/util/DLCUpstreamCursorAdapter.java | 81 +++++++++ .../ml/dlc/impl/local/MapBasedDLCImpl.java | 2 +- .../local/util/DLCUpstreamMapAdapter.java | 85 +++++++++ .../org/apache/ignite/ml/DLCPlayground.java | 39 +++-- 26 files changed, 1114 insertions(+), 276 deletions(-) create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlc/CacheDLCDatasetExample.java create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlc/LocalDLCDatasetExample.java create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlc/package-info.java rename modules/ml/src/main/java/org/apache/ignite/ml/dlc/{DLCPartitionRecoverableDataTransformer.java => DLCPartitionRecoverableTransformer.java} (58%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedTransformer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionTransformer.java rename modules/ml/src/main/java/org/apache/ignite/ml/dlc/{DLCPartitionReplicatedDataTransformer.java => DLCUpstreamEntry.java} (63%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/replicated/DLCDatasetPartitionReplicated.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/replicated/DLCLabeledDatasetPartitionReplicated.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToDatasetTransformer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToLabeledDatasetTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/recoverable/UpstreamToDatasetTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/recoverable/UpstreamToLabeledDatasetTransformer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCUpstreamCursorAdapter.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/util/DLCUpstreamMapAdapter.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/CacheDLCDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/CacheDLCDatasetExample.java new file mode 100644 index 0000000000000..d77762f5df931 --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/CacheDLCDatasetExample.java @@ -0,0 +1,111 @@ +/* + * 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.examples.ml.dlc; + +import java.util.Arrays; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.ml.dlc.DLCFactory; +import org.apache.ignite.ml.dlc.dataset.DLCDataset; +import org.apache.ignite.ml.dlc.dataset.transformer.DLCTransformers; + +/** + * How to create a DLC dataset from an existing Ignite Cache? + */ +public class CacheDLCDatasetExample { + /** Run example. */ + public static void main(String[] args) { + try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { + System.out.println(">>> D-Learn Cache Dataset example started."); + + IgniteCache persons = createCache(ignite); + + // Initialization of the d-learn context. After this step context cache will be created with partitions + // placed on the same nodes as the upstream Ignite Cache. + + // Loading of the d-learn context. During this step data will be transferred from the upstream cache to + // context cache with specified transformation (it will be performed locally because partitions are on the + // same nodes). In this case for every partition in upstream cache will be created labeled dataset partition + // and this new partition will be filled with help of specified feature and label extractors. + DLCDataset dataset = DLCFactory.createDLC( + ignite, + persons, + (data, size) -> null, + DLCTransformers.upstreamToDataset((k, v) -> { + double[] row = new double[2]; + row[0] = v.age; + row[1] = v.salary; + return row; + }, 2), + DLCDataset::new + ); + + // Calculation of the mean value. This calculation will be performed in map-reduce manner. + double[] mean = dataset.mean(); + System.out.println("Mean \n\t" + Arrays.toString(mean)); + + // Calculation of the standard deviation. This calculation will be performed in map-reduce manner. + double[] std = dataset.std(); + System.out.println("Standard deviation \n\t" + Arrays.toString(std)); + + // Calculation of the covariance matrix. This calculation will be performed in map-reduce manner. + double[][] cov = dataset.cov(); + System.out.println("Covariance matrix "); + for (double[] row : cov) + System.out.println("\t" + Arrays.toString(row)); + + // Calculation of the correlation matrix. This calculation will be performed in map-reduce manner. + double[][] corr = dataset.corr(); + System.out.println("Correlation matrix "); + for (double[] row : corr) + System.out.println("\t" + Arrays.toString(row)); + + System.out.println(">>> D-Learn Cache Dataset example completed."); + } + } + + /** */ + private static IgniteCache createCache(Ignite ignite) { + IgniteCache persons = ignite.createCache("PERSONS"); + persons.put(1, new Person("Mike", 42, 10000)); + persons.put(2, new Person("John", 32, 64000)); + persons.put(3, new Person("George", 53, 120000)); + persons.put(4, new Person("Karl", 24, 70000)); + return persons; + } + + /** */ + private static class Person { + /** */ + private final String name; + + /** */ + private final double age; + + /** */ + private final double salary; + + /** */ + public Person(String name, double age, double salary) { + this.name = name; + this.age = age; + this.salary = salary; + } + } +} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/LocalDLCDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/LocalDLCDatasetExample.java new file mode 100644 index 0000000000000..0cb95a4e327e9 --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/LocalDLCDatasetExample.java @@ -0,0 +1,112 @@ +/* + * 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.examples.ml.dlc; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.Ignite; +import org.apache.ignite.Ignition; +import org.apache.ignite.ml.dlc.DLCFactory; +import org.apache.ignite.ml.dlc.dataset.DLCDataset; +import org.apache.ignite.ml.dlc.dataset.transformer.DLCTransformers; + +/** + * How to create a DLC dataset from an existing local data? + */ +public class LocalDLCDatasetExample { + /** Run example. */ + public static void main(String[] args) { + try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { + System.out.println(">>> D-Learn Cache Dataset example started."); + + Map persons = createCache(ignite); + + // Initialization of the d-learn context. After this step context cache will be created with partitions + // placed on the same nodes as the upstream Ignite Cache. + + // Loading of the d-learn context. During this step data will be transferred from the upstream cache to + // context cache with specified transformation (it will be performed locally because partitions are on the + // same nodes). In this case for every partition in upstream cache will be created labeled dataset partition + // and this new partition will be filled with help of specified feature and label extractors. + DLCDataset dataset = DLCFactory.createDLC( + persons, + 2, + (data, size) -> null, + DLCTransformers.upstreamToDataset((k, v) -> { + double[] row = new double[2]; + row[0] = v.age; + row[1] = v.salary; + return row; + }, 2), + DLCDataset::new + ); + + // Calculation of the mean value. This calculation will be performed in map-reduce manner. + double[] mean = dataset.mean(); + System.out.println("Mean \n\t" + Arrays.toString(mean)); + + // Calculation of the standard deviation. This calculation will be performed in map-reduce manner. + double[] std = dataset.std(); + System.out.println("Standard deviation \n\t" + Arrays.toString(std)); + + // Calculation of the covariance matrix. This calculation will be performed in map-reduce manner. + double[][] cov = dataset.cov(); + System.out.println("Covariance matrix "); + for (double[] row : cov) + System.out.println("\t" + Arrays.toString(row)); + + // Calculation of the correlation matrix. This calculation will be performed in map-reduce manner. + double[][] corr = dataset.corr(); + System.out.println("Correlation matrix "); + for (double[] row : corr) + System.out.println("\t" + Arrays.toString(row)); + + System.out.println(">>> D-Learn Cache Dataset example completed."); + } + } + + /** */ + private static Map createCache(Ignite ignite) { + Map persons = new HashMap<>(); + persons.put(1, new Person("Mike", 42, 10000)); + persons.put(2, new Person("John", 32, 64000)); + persons.put(3, new Person("George", 53, 120000)); + persons.put(4, new Person("Karl", 24, 70000)); + return persons; + } + + /** */ + private static class Person { + /** */ + private final String name; + + /** */ + private final double age; + + /** */ + private final double salary; + + /** */ + public Person(String name, double age, double salary) { + this.name = name; + this.age = age; + this.salary = salary; + } + } +} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/package-info.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/package-info.java new file mode 100644 index 0000000000000..bf2c1998efece --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Machine learning Distributed Learning Context framework examples. + */ +package org.apache.ignite.examples.ml.dlc; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCFactory.java index e9a8fdf7b6403..95657ccb6f307 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCFactory.java @@ -18,7 +18,11 @@ package org.apache.ignite.ml.dlc; import java.io.Serializable; +import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.UUID; import javax.cache.Cache; import org.apache.ignite.Ignite; @@ -31,6 +35,10 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.ml.dlc.impl.cache.CacheBasedDLCImpl; import org.apache.ignite.ml.dlc.impl.cache.DLCAffinityFunctionWrapper; +import org.apache.ignite.ml.dlc.impl.cache.util.DLCUpstreamCursorAdapter; +import org.apache.ignite.ml.dlc.impl.local.MapBasedDLCImpl; +import org.apache.ignite.ml.dlc.impl.local.util.DLCUpstreamMapAdapter; +import org.apache.ignite.ml.math.functions.IgniteFunction; /** * Distributed Learning Context factory which produces contexts based on Ignite Cache and local Map. @@ -40,22 +48,25 @@ public class DLCFactory { * Constructs a new instance of Distributed Learning Context based on the specified upstream Ignite Cache and uses * Ignite Cache as reliable storage. * - * @param ignite ignite instance + * @param ignite Ignite instance * @param upstreamCache upstream cache - * @param replicatedDataLdr replicated data loader - * @param recoverableDataLdr recovered data loader + * @param replicatedTransformer replicated data transformer + * @param recoverableTransformer recoverable data transformer + * @param wrapDLC learning context wrapper * @param type of an upstream value key * @param type of an upstream value * @param type of replicated data of a partition * @param type of recoverable data of a partition - * @return Distributed Learning Context + * @param type of returned learning context + * @return distributed learning context */ @SuppressWarnings("unchecked") - public static DLC createIDD(Ignite ignite, + public static > I createDLC( + Ignite ignite, IgniteCache upstreamCache, - DLCPartitionReplicatedDataTransformer replicatedDataLdr, - DLCPartitionRecoverableDataTransformer recoverableDataLdr) { - + DLCPartitionReplicatedTransformer replicatedTransformer, + DLCPartitionRecoverableTransformer recoverableTransformer, + IgniteFunction, I> wrapDLC) { UUID dlcId = UUID.randomUUID(); AffinityFunction upstreamCacheAffinity = upstreamCache.getConfiguration(CacheConfiguration.class).getAffinity(); @@ -81,15 +92,65 @@ public static DLC> cursor = locUpstreamCache.query(qry)) { - replicatedData = replicatedDataLdr.apply(cursor, cnt); + replicated = replicatedTransformer.apply(new DLCUpstreamCursorAdapter<>(cursor), cnt); } - DLCPartition part = new DLCPartition<>(replicatedData, recoverableDataLdr); + DLCPartition part = new DLCPartition<>(replicated, recoverableTransformer); dlcCache.put(currPartIdx, part); }); } - return new CacheBasedDLCImpl<>(ignite, upstreamCache, dlcCache, dlcId); + DLC dlc = new CacheBasedDLCImpl<>(ignite, upstreamCache, dlcCache, dlcId); + + return wrapDLC.apply(dlc); + } + + /** + * Constructs a new instance of Distributed Learning Context based on the specified Map and uses local HashMap as + * reliable storage. + * + * @param upstreamData upstream data + * @param partitions number of partitions + * @param replicatedTransformer replicated data transformer + * @param recoverableTransformer recoverable data transformer + * @param wrapDLC learning context wrapper + * @param type of an upstream value key + * @param type of an upstream value + * @param type of replicated data of a partition + * @param type of recoverable data of a partition + * @param type of returned learning context + * @return distributed learning context + */ + public static > I createDLC( + Map upstreamData, int partitions, + DLCPartitionReplicatedTransformer replicatedTransformer, + DLCPartitionRecoverableTransformer recoverableTransformer, + IgniteFunction, I> wrapDLC) { + Map> dlcMap = new HashMap<>(); + + int partSize = upstreamData.size() / partitions; + + List keys = new ArrayList<>(upstreamData.keySet()); + + for (int partIdx = 0; partIdx < partitions; partIdx++) { + List partKeys = keys.subList(partIdx * partSize, Math.min((partIdx + 1) * partSize, upstreamData.size())); + Q replicated = replicatedTransformer.apply( + new DLCUpstreamMapAdapter<>(upstreamData, partKeys), + (long) partKeys.size() + ); + W recoverable = recoverableTransformer.apply( + new DLCUpstreamMapAdapter<>(upstreamData, partKeys), + (long) partKeys.size(), + replicated + ); + DLCPartition part = new DLCPartition<>(replicated, null); + part.setRecoverableData(recoverable); + dlcMap.put(partIdx, part); + } + + DLC dlc = new MapBasedDLCImpl<>(dlcMap, partitions); + + return wrapDLC.apply(dlc); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartition.java index 6ddc75d810dfd..d7630aa9968a7 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartition.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartition.java @@ -38,7 +38,7 @@ public class DLCPartition private final Q replicatedData; /** Loader of the recoverable part of this partition. */ - private final DLCPartitionRecoverableDataTransformer recoverableDataTransformer; + private final DLCPartitionRecoverableTransformer recoverableDataTransformer; /** Recoverable data. */ private transient W recoverableData; @@ -49,8 +49,7 @@ public class DLCPartition * @param replicatedData replicated data * @param recoverableDataTransformer transformer of the recoverable part of this partition */ - public DLCPartition(Q replicatedData, - DLCPartitionRecoverableDataTransformer recoverableDataTransformer) { + public DLCPartition(Q replicatedData, DLCPartitionRecoverableTransformer recoverableDataTransformer) { this.replicatedData = replicatedData; this.recoverableDataTransformer = recoverableDataTransformer; } @@ -61,7 +60,7 @@ public Q getReplicatedData() { } /** */ - public DLCPartitionRecoverableDataTransformer getRecoverableDataTransformer() { + public DLCPartitionRecoverableTransformer getRecoverableDataTransformer() { return recoverableDataTransformer; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableDataTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableTransformer.java similarity index 58% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableDataTransformer.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableTransformer.java index f7a7ac1696b28..94c4a03566e97 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableDataTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableTransformer.java @@ -17,21 +17,25 @@ package org.apache.ignite.ml.dlc; -import java.io.Serializable; -import javax.cache.Cache; import org.apache.ignite.ml.math.functions.IgniteTriFunction; /** - * Transformer which transforms upstream data to any desired type of recoverable data. * - * @param type of an upstream value key - * @param type of an upstream value - * @param type of replicated data of a partition - * @param type of recoverable data of a partition + * + * @param + * @param + * @param + * @param */ @FunctionalInterface -public interface DLCPartitionRecoverableDataTransformer - extends IgniteTriFunction>, Long, Q, W> { - /** {@inheritDoc} */ - @Override public W apply(Iterable> upstreamData, Long upstreamDataSize, Q replicatedData); +public interface DLCPartitionRecoverableTransformer + extends IgniteTriFunction>, Long, Q, W> { + /** + * + * @param upstreamData + * @param upstreamDataSize + * @param replicatedData + * @return + */ + W apply(Iterable> upstreamData, Long upstreamDataSize, Q replicatedData); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedTransformer.java new file mode 100644 index 0000000000000..241a41d61ff36 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedTransformer.java @@ -0,0 +1,40 @@ +/* + * 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.ml.dlc; + +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +/** + * + * + * @param + * @param + * @param + */ +@FunctionalInterface +public interface DLCPartitionReplicatedTransformer + extends IgniteBiFunction>, Long, Q> { + /** + * + * + * @param upstreamData + * @param upstreamDataSize + * @return + */ + Q apply(Iterable> upstreamData, Long upstreamDataSize); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionTransformer.java deleted file mode 100644 index 42165d330614c..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionTransformer.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.ml.dlc; - -import java.io.Serializable; -import javax.cache.Cache; - -/** - * Distributed Learning Dataset partition transformer. - * - * @param type of an upstream value key - * @param type of an upstream value - * @param type of replicated data of a partition - * @param type of recoverable data of a partition - */ -public interface DLCPartitionTransformer> extends Serializable { - /** - * Transformer which transforms upstream data to any desired type of recoverable data. - * - * @param upstreamData upstream data - * @param upstreamDataSize upstream data size - * @param replicatedData replicated data - * @return recoverable data - */ - public W transformRecoverablePart(Iterable> upstreamData, Long upstreamDataSize, Q replicatedData); - - /** - * Transformer which transforms upstream data to any desired type of replicated data. - * - * @param upstreamData upstream data - * @param upstreamDataSize upstream data size - * @return replicated data - */ - public Q transformReplicatedPart(Iterable> upstreamData, Long upstreamDataSize); - - /** - * Wraps DLC to provide partition-specific API. - * - * @param ctx distributed learning context - * @return wrapped context - */ - public I wrapDLC(DLC ctx); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedDataTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCUpstreamEntry.java similarity index 63% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedDataTransformer.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCUpstreamEntry.java index 9bf0d18df26fb..0e8ff5cbbed26 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedDataTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCUpstreamEntry.java @@ -17,20 +17,37 @@ package org.apache.ignite.ml.dlc; -import java.io.Serializable; -import javax.cache.Cache; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; - /** - * Transformer which transforms upstream data to any desired type of replicated data. + * Distributed Learning Context upstream entry. * * @param type of an upstream value key * @param type of an upstream value - * @param type of replicated data */ -@FunctionalInterface -public interface DLCPartitionReplicatedDataTransformer - extends IgniteBiFunction>, Long, Q> { - /** {@inheritDoc} */ - @Override public Q apply(Iterable> upstreamData, Long upstreamDataSize); +public class DLCUpstreamEntry { + /** Key. */ + private final K key; + + /** Value. */ + private final V val; + + /** + * Constructs a new instance of upstream entry. + * + * @param key key + * @param val value + */ + public DLCUpstreamEntry(K key, V val) { + this.key = key; + this.val = val; + } + + /** */ + public K getKey() { + return key; + } + + /** */ + public V getValue() { + return val; + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCDataset.java index 952b754dbbfad..c8ecec6d9ba20 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCDataset.java @@ -17,25 +17,179 @@ package org.apache.ignite.ml.dlc.dataset; -import java.io.Serializable; +import com.github.fommil.netlib.BLAS; import org.apache.ignite.ml.dlc.DLC; import org.apache.ignite.ml.dlc.dataset.part.recoverable.DLCDatasetPartitionRecoverable; +import org.apache.ignite.ml.dlc.dataset.part.replicated.DLCDatasetPartitionReplicated; /** * Dataset provides API to work with dataset. * * @param type of an upstream value key * @param type of an upstream value - * @param type of replicated data of a partition */ -public class DLCDataset extends DLCWrapper { +public class DLCDataset extends DLCWrapper { + /** BLAS (Basic Linear Algebra Subprograms) instance. */ + private static final BLAS blas = BLAS.getInstance(); /** * Constructs a new instance of Distributed Learning Context wrapper * * @param delegate delegate which actually performs base functions */ public DLCDataset( - DLC delegate) { + DLC delegate) { super(delegate); } + + /** + * Calculates mean value by all columns. + * + * @return mean values + */ + public double[] mean() { + ValueWithCount res = delegate.compute((part, partIdx) -> { + DLCDatasetPartitionRecoverable recoverableData = part.getRecoverableData(); + + double[] features = recoverableData.getFeatures(); + int rows = recoverableData.getRows(); + int cols = recoverableData.getCols(); + + double[] y = new double[cols]; + + for (int col = 0; col < cols; col++) + for (int j = col * rows; j < (col + 1) * rows; j++) + y[col] += features[j]; + + return new ValueWithCount<>(y, rows); + }, (a, b) -> a == null ? b : b == null ? a : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); + + if (res != null) { + blas.dscal(res.val.length, 1.0 / res.cnt, res.val, 1); + return res.val; + } + + return null; + } + + /** + * Calculates standard deviation by all columns. + * + * @return standard deviations + */ + public double[] std() { + double[] mean = mean(); + ValueWithCount res = delegate.compute(part -> { + DLCDatasetPartitionRecoverable recoverableData = part.getRecoverableData(); + + double[] features = recoverableData.getFeatures(); + int rows = recoverableData.getRows(); + int cols = recoverableData.getCols(); + + double[] y = new double[cols]; + + for (int col = 0; col < cols; col++) + for (int j = col * rows; j < (col + 1) * rows; j++) + y[col] += Math.pow(features[j] - mean[col], 2); + + return new ValueWithCount<>(y, rows); + }, (a, b) -> a == null ? b : b == null ? a : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); + + if (res != null) { + blas.dscal(res.val.length, 1.0 / res.cnt, res.val, 1); + for (int i = 0; i < res.val.length; i++) + res.val[i] = Math.sqrt(res.val[i]); + return res.val; + } + + return null; + } + + /** + * Calculates covariance matrix by all columns. + * + * @return covariance matrix + */ + public double[][] cov() { + double[] mean = mean(); + ValueWithCount res = delegate.compute(part -> { + DLCDatasetPartitionRecoverable recoverableData = part.getRecoverableData(); + + double[] features = recoverableData.getFeatures(); + int rows = recoverableData.getRows(); + int cols = recoverableData.getCols(); + + double[][] y = new double[cols][cols]; + + for (int firstCol = 0; firstCol < cols; firstCol++) + for (int secondCol = 0; secondCol < cols; secondCol++) { + + for (int k = 0; k < rows; k++) { + double firstVal = features[rows * firstCol + k]; + double secondVal = features[rows * secondCol + k]; + y[firstCol][secondCol] += ((firstVal - mean[firstCol]) * (secondVal - mean[secondCol])); + } + } + + return new ValueWithCount<>(y, rows); + }, (a, b) -> a == null ? b : b == null ? a : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); + + return res != null ? scale(res.val, 1.0 / res.cnt) : null; + } + + /** + * Calculates correlation matrix by all columns. + * + * @return correlation matrix + */ + public double[][] corr() { + double[][] cov = cov(); + double[] std = std(); + + for (int i = 0; i < cov.length; i++) + for (int j = 0; j < cov[0].length; j++) + cov[i][j] /= (std[i]*std[j]); + + return cov; + } + + /** */ + private static double[] sum(double[] a, double[] b) { + for (int i = 0; i < a.length; i++) + a[i] += b[i]; + + return a; + } + + /** */ + private static double[][] sum(double[][] a, double[][] b) { + for (int i = 0; i < a.length; i++) + for (int j = 0; j < a[i].length; j++) + a[i][j] += b[i][j]; + + return a; + } + + /** */ + private static double[][] scale(double[][] a, double alpha) { + for (int i = 0; i < a.length; i++) + for (int j = 0; j < a[i].length; j++) + a[i][j] *= alpha; + + return a; + } + + /** */ + private static class ValueWithCount { + /** */ + private final V val; + + /** */ + private final int cnt; + + /** */ + ValueWithCount(V val, int cnt) { + this.val = val; + this.cnt = cnt; + } + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCLabeledDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCLabeledDataset.java index f64151c83222b..3a44203b0d389 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCLabeledDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCLabeledDataset.java @@ -17,26 +17,25 @@ package org.apache.ignite.ml.dlc.dataset; -import java.io.Serializable; import org.apache.ignite.ml.dlc.DLC; import org.apache.ignite.ml.dlc.dataset.part.recoverable.DLCLabeledDatasetPartitionRecoverable; +import org.apache.ignite.ml.dlc.dataset.part.replicated.DLCLabeledDatasetPartitionReplicated; /** * Dataset provides API to work with labeled dataset. * * @param type of an upstream value key * @param type of an upstream value - * @param type of replicated data of a partition */ -public class DLCLabeledDataset - extends DLCWrapper { +public class DLCLabeledDataset + extends DLCWrapper { /** * Constructs a new instance of Distributed Learning Context wrapper * * @param delegate delegate which actually performs base functions */ public DLCLabeledDataset( - DLC delegate) { + DLC delegate) { super(delegate); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCWrapper.java index f04470eacc7e5..bb44d597e89e6 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCWrapper.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCWrapper.java @@ -34,7 +34,7 @@ */ public class DLCWrapper implements DLC { /** Delegate which actually performs base functions like {@code compute()} and {@code close()}. */ - private final DLC delegate; + protected final DLC delegate; /** * Constructs a new instance of Distributed Learning Context wrapper diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCDatasetPartitionRecoverable.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCDatasetPartitionRecoverable.java index 9441f11793371..6fdc5b82b1259 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCDatasetPartitionRecoverable.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCDatasetPartitionRecoverable.java @@ -17,19 +17,49 @@ package org.apache.ignite.ml.dlc.dataset.part.recoverable; +/** + * Recoverable part of a DLC dataset partition contains matrix with features in a dense flat column-major format. + */ public class DLCDatasetPartitionRecoverable implements AutoCloseable { + /** Matrix with features in a dense flat column-major format. */ + private final double[] features; + + /** Number of rows. */ + private final int rows; - private final double[][] data; + /** Number of columns. */ + private final int cols; - public DLCDatasetPartitionRecoverable(double[][] data) { - this.data = data; + /** + * Constructs a new instance of recoverable data of DLC dataset partition. + * + * @param features matrix with features in a dense flat column-major format + * @param rows number of rows + * @param cols number of columns + */ + public DLCDatasetPartitionRecoverable(double[] features, int rows, int cols) { + this.features = features; + this.rows = rows; + this.cols = cols; } - public double[][] getData() { - return data; + /** */ + public double[] getFeatures() { + return features; } - @Override public void close() { + /** */ + public int getRows() { + return rows; + } + /** */ + public int getCols() { + return cols; + } + + /** {@inheritDoc} */ + @Override public void close() { + // do nothing } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCLabeledDatasetPartitionRecoverable.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCLabeledDatasetPartitionRecoverable.java index ebb9c62bd3ef4..1c85df865c361 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCLabeledDatasetPartitionRecoverable.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCLabeledDatasetPartitionRecoverable.java @@ -17,16 +17,65 @@ package org.apache.ignite.ml.dlc.dataset.part.recoverable; -public class DLCLabeledDatasetPartitionRecoverable extends DLCDatasetPartitionRecoverable { +/** + * Recoverable part of a DLC labeled dataset partition contains matrix of features in a dense flat column-major format + * and vector of values. + */ +public class DLCLabeledDatasetPartitionRecoverable implements AutoCloseable { + /** Matrix of features in a dense flat column-major format. */ + private final double[] features; + + /** Number of rows. */ + private final int rows; + /** Number of columns. */ + private final int cols; + + /** Vector with labels. */ private double[] labels; - public DLCLabeledDatasetPartitionRecoverable(double[][] data, double[] labels) { - super(data); + /** + * Constructs a new instance of recoverable data of DLC labeled dataset partition. + * + * @param features matrix of features in a dense flat column-major format + * @param rows number of rows + * @param cols number of columns + * @param labels vector with labels + */ + public DLCLabeledDatasetPartitionRecoverable(double[] features, int rows, int cols, double[] labels) { + this.features = features; + this.rows = rows; + this.cols = cols; this.labels = labels; } + /** */ + public double[] getFeatures() { + return features; + } + + /** */ + public int getRows() { + return rows; + } + + /** */ + public int getCols() { + return cols; + } + + /** */ public double[] getLabels() { return labels; } + + /** */ + public void setLabels(double[] labels) { + this.labels = labels; + } + + /** {@inheritDoc} */ + @Override public void close() { + // do nothing + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/replicated/DLCDatasetPartitionReplicated.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/replicated/DLCDatasetPartitionReplicated.java new file mode 100644 index 0000000000000..38623a37b4a42 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/replicated/DLCDatasetPartitionReplicated.java @@ -0,0 +1,28 @@ +/* + * 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.ml.dlc.dataset.part.replicated; + +import java.io.Serializable; + +/** + * Replicated part of DLC dataset partition. + */ +public class DLCDatasetPartitionReplicated implements Serializable { + /** */ + private static final long serialVersionUID = 3579701425183018483L; +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/replicated/DLCLabeledDatasetPartitionReplicated.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/replicated/DLCLabeledDatasetPartitionReplicated.java new file mode 100644 index 0000000000000..9cc34917365df --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/replicated/DLCLabeledDatasetPartitionReplicated.java @@ -0,0 +1,28 @@ +/* + * 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.ml.dlc.dataset.part.replicated; + +import java.io.Serializable; + +/** + * Replicated part of DLC labeled dataset partition. + */ +public class DLCLabeledDatasetPartitionReplicated implements Serializable { + /** */ + private static final long serialVersionUID = 132087952507878026L; +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/DLCTransformers.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/DLCTransformers.java index a2970cfb1f208..fa17c9315519f 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/DLCTransformers.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/DLCTransformers.java @@ -17,23 +17,46 @@ package org.apache.ignite.ml.dlc.dataset.transformer; -import javax.cache.Cache; -import org.apache.ignite.ml.math.functions.IgniteFunction; +import org.apache.ignite.ml.dlc.dataset.DLCDataset; +import org.apache.ignite.ml.dlc.dataset.DLCLabeledDataset; +import org.apache.ignite.ml.dlc.dataset.transformer.recoverable.UpstreamToDatasetTransformer; +import org.apache.ignite.ml.dlc.dataset.transformer.recoverable.UpstreamToLabeledDatasetTransformer; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; /** * Aggregator which allows to find desired transformer from one partition data type to another. This class doesn't * introduce a new functionality, but helps to work efficiently with existing transformers. */ public class DLCTransformers { - + /** + * Creates a new instance of transformer which transforms upstream data into the {@link DLCDataset} using the + * specified feature extractor. + * + * @param featureExtractor feature extractor + * @param features number of features + * @param type of an upstream value key + * @param type of an upstream value + * @return transformer + */ public static UpstreamToDatasetTransformer upstreamToDataset( - IgniteFunction, double[]> featureExtractor, int features) { + IgniteBiFunction featureExtractor, int features) { return new UpstreamToDatasetTransformer<>(featureExtractor, features); } + /** + * Creates a new instance of transformer which transforms upstream data into the {@link DLCLabeledDataset} using the + * specified feature and label extractors. + * + * @param featureExtractor feature extractor + * @param lbExtractor label extractor + * @param features number of features + * @param type of an upstream value key + * @param type of an upstream value + * @return transformer + */ public static UpstreamToLabeledDatasetTransformer upstreamToLabeledDataset( - IgniteFunction, double[]> featureExtractor, - IgniteFunction, Double> lbExtractor, int features) { + IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor, int features) { return new UpstreamToLabeledDatasetTransformer<>(featureExtractor, lbExtractor, features); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToDatasetTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToDatasetTransformer.java deleted file mode 100644 index c48c51a42fd8f..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToDatasetTransformer.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.ml.dlc.dataset.transformer; - -import java.io.Serializable; -import javax.cache.Cache; -import org.apache.ignite.ml.dlc.DLCPartitionRecoverableDataTransformer; -import org.apache.ignite.ml.dlc.dataset.part.recoverable.DLCDatasetPartitionRecoverable; -import org.apache.ignite.ml.math.functions.IgniteFunction; - -/** - * Transforms upstream data into dataset recoverable part of partition using specified feature extractor. - * - * @param type of an upstream value key - * @param type of an upstream value - */ -public class UpstreamToDatasetTransformer implements DLCPartitionRecoverableDataTransformer { - /** */ - private static final long serialVersionUID = -3713681392540367983L; - - /** Feature extractor. */ - private final IgniteFunction, double[]> featureExtractor; - - /** Number of features. */ - private final int features; - - /** - * Constructs a new instance of transformer. - * - * @param featureExtractor feature extractor - * @param features number of features - */ - protected UpstreamToDatasetTransformer( - IgniteFunction, double[]> featureExtractor, int features) { - this.featureExtractor = featureExtractor; - this.features = features; - } - - /** {@inheritDoc} */ - @Override public DLCDatasetPartitionRecoverable apply(Iterable> entries, Long aLong, Serializable serializable) { - double[][] features = new double[Math.toIntExact(aLong)][]; - int ptr = 0; - for (Cache.Entry e : entries) - features[ptr++] = featureExtractor.apply(e); - return new DLCDatasetPartitionRecoverable(features); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToLabeledDatasetTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToLabeledDatasetTransformer.java deleted file mode 100644 index f4ebe50128fa0..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToLabeledDatasetTransformer.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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.ml.dlc.dataset.transformer; - -import java.io.Serializable; -import javax.cache.Cache; -import org.apache.ignite.ml.dlc.DLCPartitionRecoverableDataTransformer; -import org.apache.ignite.ml.dlc.dataset.part.recoverable.DLCLabeledDatasetPartitionRecoverable; -import org.apache.ignite.ml.math.functions.IgniteFunction; - -/** - * Transforms upstream data into labeled dataset recoverable part of partition using specified feature extractor and - * label extractor. - * - * @param type of an upstream value key - * @param type of an upstream value - */ -public class UpstreamToLabeledDatasetTransformer implements DLCPartitionRecoverableDataTransformer { - /** */ - private static final long serialVersionUID = -1224768715207401297L; - - /** Feature extractor. */ - private final IgniteFunction, double[]> featureExtractor; - - /** Label extractor. */ - private final IgniteFunction, Double> lbExtractor; - - /** Number of features. */ - private final int features; - - /** - * Constructs a new instance of transformer. - * - * @param featureExtractor feature extractor - * @param lbExtractor label extractor - * @param features number of features - */ - protected UpstreamToLabeledDatasetTransformer( - IgniteFunction, double[]> featureExtractor, - IgniteFunction, Double> lbExtractor, int features) { - this.featureExtractor = featureExtractor; - this.lbExtractor = lbExtractor; - this.features = features; - } - - /** {@inheritDoc} */ - @Override public DLCLabeledDatasetPartitionRecoverable apply(Iterable> entries, Long aLong, - Serializable ser) { - double[][] features = new double[Math.toIntExact(aLong)][]; - double[] labels = new double[Math.toIntExact(aLong)]; - int ptr = 0; - for (Cache.Entry e : entries) { - features[ptr] = featureExtractor.apply(e); - labels[ptr] = lbExtractor.apply(e); - ptr++; - } - return new DLCLabeledDatasetPartitionRecoverable(features, labels); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/recoverable/UpstreamToDatasetTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/recoverable/UpstreamToDatasetTransformer.java new file mode 100644 index 0000000000000..46f872afe0f79 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/recoverable/UpstreamToDatasetTransformer.java @@ -0,0 +1,84 @@ +/* + * 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.ml.dlc.dataset.transformer.recoverable; + +import org.apache.ignite.ml.dlc.DLCPartitionRecoverableTransformer; +import org.apache.ignite.ml.dlc.DLCUpstreamEntry; +import org.apache.ignite.ml.dlc.dataset.DLCDataset; +import org.apache.ignite.ml.dlc.dataset.part.recoverable.DLCDatasetPartitionRecoverable; +import org.apache.ignite.ml.dlc.dataset.part.replicated.DLCDatasetPartitionReplicated; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +/** + * Transforms upstream data into the {@link DLCDataset} using the specified feature extractor. + * + * @param type of an upstream value key + * @param type of an upstream value + */ +public class UpstreamToDatasetTransformer + implements DLCPartitionRecoverableTransformer { + /** */ + private static final long serialVersionUID = -3713681392540367983L; + + /** Feature extractor. */ + private final IgniteBiFunction featureExtractor; + + /** Number of features. */ + private final int features; + + /** + * Constructs a new instance of transformer. + * + * @param featureExtractor feature extractor + * @param features number of features + */ + public UpstreamToDatasetTransformer( + IgniteBiFunction featureExtractor, int features) { + this.featureExtractor = featureExtractor; + this.features = features; + } + + /** + * Transforms upstream data to {@link DLCDatasetPartitionRecoverable}. + * + * @param upstreamData upstream data + * @param upstreamDataSize upstream data size + * @param replicatedData replicated data + * @return dataset partition recoverable data + */ + @Override public DLCDatasetPartitionRecoverable apply(Iterable> upstreamData, Long upstreamDataSize, + DLCDatasetPartitionReplicated replicatedData) { + int rows = Math.toIntExact(upstreamDataSize), cols = features; + + double[] features = new double[rows * cols]; + + int ptr = 0; + for (DLCUpstreamEntry e : upstreamData) { + double[] row = featureExtractor.apply(e.getKey(), e.getValue()); + + assert cols == row.length; + + for (int i = 0; i < cols; i++) + features[i * rows + ptr] = row[i]; + + ptr++; + } + + return new DLCDatasetPartitionRecoverable(features, rows, cols); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/recoverable/UpstreamToLabeledDatasetTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/recoverable/UpstreamToLabeledDatasetTransformer.java new file mode 100644 index 0000000000000..12345ac256fd9 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/recoverable/UpstreamToLabeledDatasetTransformer.java @@ -0,0 +1,94 @@ +/* + * 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.ml.dlc.dataset.transformer.recoverable; + +import org.apache.ignite.ml.dlc.DLCPartitionRecoverableTransformer; +import org.apache.ignite.ml.dlc.DLCUpstreamEntry; +import org.apache.ignite.ml.dlc.dataset.DLCLabeledDataset; +import org.apache.ignite.ml.dlc.dataset.part.recoverable.DLCLabeledDatasetPartitionRecoverable; +import org.apache.ignite.ml.dlc.dataset.part.replicated.DLCLabeledDatasetPartitionReplicated; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +/** + * Transforms upstream data into the {@link DLCLabeledDataset} using the specified feature and label extractors. + * + * @param type of an upstream value key + * @param type of an upstream value + */ +public class UpstreamToLabeledDatasetTransformer + implements DLCPartitionRecoverableTransformer { + /** */ + private static final long serialVersionUID = -1224768715207401297L; + + /** Feature extractor. */ + private final IgniteBiFunction featureExtractor; + + /** Label extractor. */ + private final IgniteBiFunction lbExtractor; + + /** Number of features. */ + private final int features; + + /** + * Constructs a new instance of transformer. + * + * @param featureExtractor feature extractor + * @param lbExtractor label extractor + * @param features number of features + */ + public UpstreamToLabeledDatasetTransformer( + IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor, int features) { + this.featureExtractor = featureExtractor; + this.lbExtractor = lbExtractor; + this.features = features; + } + + /** + * Transforms upstream data to {@link DLCLabeledDatasetPartitionRecoverable}. + * + * @param upstreamData upstream data + * @param upstreamDataSize upstream data size + * @param replicatedData replicated data + * @return labeled dataset recoverable data + */ + @Override public DLCLabeledDatasetPartitionRecoverable apply(Iterable> upstreamData, + Long upstreamDataSize, DLCLabeledDatasetPartitionReplicated replicatedData) { + int rows = Math.toIntExact(upstreamDataSize), cols = features; + + double[] features = new double[rows * cols]; + double[] labels = new double[rows]; + + int ptr = 0; + for (DLCUpstreamEntry e : upstreamData) { + double[] row = featureExtractor.apply(e.getKey(), e.getValue()); + + assert cols == row.length; + + for (int i = 0; i < cols; i++) + features[i * rows + ptr] = row[i]; + + labels[ptr] = lbExtractor.apply(e.getKey(), e.getValue()); + + ptr++; + } + + return new DLCLabeledDatasetPartitionRecoverable(features, rows, cols, labels); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/CacheBasedDLCPartitionBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/CacheBasedDLCPartitionBuilder.java index 6ea13e7dbb99c..3cc40a774471d 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/CacheBasedDLCPartitionBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/CacheBasedDLCPartitionBuilder.java @@ -35,7 +35,8 @@ import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.ml.dlc.DLCPartition; -import org.apache.ignite.ml.dlc.DLCPartitionRecoverableDataTransformer; +import org.apache.ignite.ml.dlc.DLCUpstreamEntry; +import org.apache.ignite.ml.math.functions.IgniteTriFunction; /** * Distributed Learning Context partition builder which constructs a partition from two parts: replicated data which is @@ -140,7 +141,8 @@ public DLCPartition build() { * @param replicatedData replicated data * @return recoverable data */ - private W loadRecoverableData(DLCPartitionRecoverableDataTransformer recoverableDataTransformer, + private W loadRecoverableData( + IgniteTriFunction>, Long, Q, W> recoverableDataTransformer, Q replicatedData) { Affinity upstreamCacheAffinity = ignite.affinity(upstreamCacheName); ClusterNode partNode = upstreamCacheAffinity.mapPartitionToNode(partIdx); @@ -160,7 +162,7 @@ private W loadRecoverableData(DLCPartitionRecoverableDataTransformer // TODO: how to guarantee that cache size will not be changed between these calls? long cnt = locUpstreamCache.localSizeLong(partIdx); try (QueryCursor> cursor = locUpstreamCache.query(qry)) { - return recoverableDataTransformer.apply(cursor, cnt, replicatedData); + return recoverableDataTransformer.apply(new DLCUpstreamCursorAdapter<>(cursor), cnt, replicatedData); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCUpstreamCursorAdapter.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCUpstreamCursorAdapter.java new file mode 100644 index 0000000000000..8bb0fc8607c47 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCUpstreamCursorAdapter.java @@ -0,0 +1,81 @@ +/* + * 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.ml.dlc.impl.cache.util; + +import java.util.Iterator; +import javax.cache.Cache; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.ml.dlc.DLCUpstreamEntry; + +/** + * Cursor adapter used to transform {@link Cache.Entry} received from Ignite Cache query cursor into DLC-specific + * {@link DLCUpstreamEntry}. + * + * @param type of an upstream value key + * @param type of an upstream value + */ +public class DLCUpstreamCursorAdapter implements Iterable> { + /** Ignite Cache cursor. */ + private final QueryCursor> cursor; + + /** + * Constructs a new instance of query cursor adapter. + * + * @param cursor Ignite Cache query cursor + */ + public DLCUpstreamCursorAdapter(QueryCursor> cursor) { + this.cursor = cursor; + } + + /** {@inheritDoc} */ + @Override public Iterator> iterator() { + return new DLCUpstreamCursorWrapperIterator(cursor.iterator()); + } + + /** + * Utils class representing iterator of {@link DLCUpstreamEntry} based on iterator of {@link Cache.Entry}. + */ + private final class DLCUpstreamCursorWrapperIterator implements Iterator> { + /** Cache entry iterator. */ + private final Iterator> delegate; + + /** + * Constructs a new instance of iterator. + * + * @param delegate cache entry iterator + */ + DLCUpstreamCursorWrapperIterator(Iterator> delegate) { + this.delegate = delegate; + } + + /** {@inheritDoc} */ + @Override public boolean hasNext() { + return delegate.hasNext(); + } + + /** {@inheritDoc} */ + @Override public DLCUpstreamEntry next() { + Cache.Entry next = delegate.next(); + + if (next == null) + return null; + + return new DLCUpstreamEntry<>(next.getKey(), next.getValue()); + } + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCImpl.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCImpl.java index 0781ef812cdf0..11a82dc8105f8 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCImpl.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCImpl.java @@ -35,7 +35,7 @@ * @param type of replicated data of a partition * @param type of recoverable data of a partition */ -public class MapBasedDLCImpl, V, Q extends Serializable, W extends AutoCloseable> +public class MapBasedDLCImpl implements DLC { /** Map containing pairs of partition index and partitions. */ private final Map> dlcMap; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/util/DLCUpstreamMapAdapter.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/util/DLCUpstreamMapAdapter.java new file mode 100644 index 0000000000000..b3622058db1b7 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/util/DLCUpstreamMapAdapter.java @@ -0,0 +1,85 @@ +/* + * 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.ml.dlc.impl.local.util; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.ignite.ml.dlc.DLCUpstreamEntry; + +/** + * This is {@code Map} adapter used to transform {@code Map} entries into DLC-specific {@link DLCUpstreamEntry}. + * + * @param type of an upstream value key + * @param type of an upstream value + */ +public class DLCUpstreamMapAdapter implements Iterable> { + /** Upstream data {@code Map}. */ + private final Map upstreamData; + + /** Ordered upstream data keys. */ + private final List keys; + + /** + * Constructs a new instance of {@code Map} adapter. + * + * @param upstreamData upstream data {@code Map} + * @param keys ordered upstream data keys + */ + public DLCUpstreamMapAdapter(Map upstreamData, List keys) { + this.upstreamData = upstreamData; + this.keys = keys; + } + + /** {@inheritDoc} */ + @Override public Iterator> iterator() { + return new DLCUpstreamKeysWrapperIterator(keys.iterator()); + } + + /** + * Utils class representing iterator of {@link DLCUpstreamEntry} based on iterator of {@code Map} keys. + */ + private final class DLCUpstreamKeysWrapperIterator implements Iterator> { + /** Keys iterator. */ + private final Iterator keysIterator; + + /** + * Constructs a new instance of iterator. + * + * @param keysIterator keys iterator + */ + DLCUpstreamKeysWrapperIterator(Iterator keysIterator) { + this.keysIterator = keysIterator; + } + + /** {@inheritDoc} */ + @Override public boolean hasNext() { + return keysIterator.hasNext(); + } + + /** {@inheritDoc} */ + @Override public DLCUpstreamEntry next() { + K nextKey = keysIterator.next(); + + if (nextKey == null) + return null; + + return new DLCUpstreamEntry<>(nextKey, upstreamData.get(nextKey)); + } + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java b/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java index 1aa954bb28dfe..c7f373f7fc77f 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java @@ -1,15 +1,13 @@ package org.apache.ignite.ml; -import java.io.Serializable; import java.util.Arrays; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.dlc.DLC; import org.apache.ignite.ml.dlc.DLCFactory; -import org.apache.ignite.ml.dlc.dataset.part.recoverable.DLCDatasetPartitionRecoverable; +import org.apache.ignite.ml.dlc.dataset.DLCDataset; import org.apache.ignite.ml.dlc.dataset.transformer.DLCTransformers; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -51,21 +49,32 @@ public void testTest() { for (int i = 0; i < 40; i++) cache.put(i, i); - DLC idd = DLCFactory.createIDD( + DLCDataset dataset = DLCFactory.createDLC( ignite, cache, - (iter, cnt) -> null, - DLCTransformers.upstreamToDataset(e -> new double[]{1, 2, 3}, 3) + (data, size) -> null, + DLCTransformers.upstreamToDataset((k, v) -> new double[]{1, 2, 3}, 3), + DLCDataset::new ); - for (int i = 0; i < 3; i++) { - idd.compute(part -> { - StringBuilder builder = new StringBuilder(); - builder.append("PART \n"); - for (double[] row : part.getRecoverableData().getData()) - builder.append(Arrays.toString(row)).append("\n"); - System.err.println(builder.toString()); - }); - } + // Calculation of the mean value. This calculation will be performed in map-reduce manner. + double[] mean = dataset.mean(); + System.out.println("Mean \n\t" + Arrays.toString(mean)); + + // Calculation of the standard deviation. This calculation will be performed in map-reduce manner. + double[] std = dataset.std(); + System.out.println("Standard deviation \n\t" + Arrays.toString(std)); + + // Calculation of the covariance matrix. This calculation will be performed in map-reduce manner. + double[][] cov = dataset.cov(); + System.out.println("Covariance matrix "); + for (double[] row : cov) + System.out.println("\t" + Arrays.toString(row)); + + // Calculation of the correlation matrix. This calculation will be performed in map-reduce manner. + double[][] corr = dataset.corr(); + System.out.println("Correlation matrix "); + for (double[] row : corr) + System.out.println("\t" + Arrays.toString(row)); } } From c21e43406a1a1d398277bc40a818d87752afb11a Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Thu, 25 Jan 2018 20:03:53 +0300 Subject: [PATCH 17/38] IGNITE-7437 Update DLC and add examples. --- .../misc/deployment/DeploymentExample.java | 6 +- .../ml/dlc/CacheDLCDatasetExample.java | 27 ++-- .../ml/dlc/LocalDLCDatasetExample.java | 20 +-- .../transformer/AlgorithmSpecificDataset.java | 97 +++++++++++++ .../AlgorithmSpecificRecoverableData.java | 73 ++++++++++ ...thmSpecificRecoverableDataTransformer.java | 45 ++++++ .../AlgorithmSpecificReplicatedData.java | 21 ++- ...ithmSpecificReplicatedDataTransformer.java | 20 ++- .../transformer/DLCTransformerExample.java | 108 +++++++++++++++ .../org/apache/ignite/ml/dlc/DLCFactory.java | 128 +++--------------- .../apache/ignite/ml/dlc/DLCPartition.java | 3 +- .../DLCPartitionRecoverableTransformer.java | 40 ++++-- .../DLCPartitionReplicatedTransformer.java | 33 +++-- .../ignite/ml/dlc/dataset/DLCDataset.java | 9 +- .../ml/dlc/dataset/DLCLabeledDataset.java | 10 +- .../DLCDatasetPartitionRecoverable.java | 2 +- ...DLCLabeledDatasetPartitionRecoverable.java | 2 +- .../dataset/transformer/DLCTransformers.java | 8 +- .../UpstreamToDatasetTransformer.java | 14 +- .../UpstreamToLabeledDatasetTransformer.java | 15 +- .../dlc/impl/cache/CacheBasedDLCFactory.java | 109 +++++++++++++++ .../util/CacheBasedDLCPartitionBuilder.java | 7 +- .../DLCAffinityFunctionWrapper.java | 2 +- .../ml/dlc/impl/local/MapBasedDLCFactory.java | 88 ++++++++++++ .../org/apache/ignite/ml/DLCPlayground.java | 15 +- 25 files changed, 683 insertions(+), 219 deletions(-) create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificDataset.java create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificRecoverableData.java create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificRecoverableDataTransformer.java rename modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/replicated/DLCDatasetPartitionReplicated.java => examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificReplicatedData.java (62%) rename modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/replicated/DLCLabeledDatasetPartitionReplicated.java => examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificReplicatedDataTransformer.java (50%) create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/DLCTransformerExample.java rename modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/{recoverable => }/DLCDatasetPartitionRecoverable.java (97%) rename modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/{recoverable => }/DLCLabeledDatasetPartitionRecoverable.java (97%) rename modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/{recoverable => }/UpstreamToDatasetTransformer.java (81%) rename modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/{recoverable => }/UpstreamToLabeledDatasetTransformer.java (82%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/CacheBasedDLCFactory.java rename modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/{ => util}/DLCAffinityFunctionWrapper.java (98%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCFactory.java diff --git a/examples/src/main/java/org/apache/ignite/examples/misc/deployment/DeploymentExample.java b/examples/src/main/java/org/apache/ignite/examples/misc/deployment/DeploymentExample.java index 9497ed7b0e879..b42b0c991a30d 100644 --- a/examples/src/main/java/org/apache/ignite/examples/misc/deployment/DeploymentExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/misc/deployment/DeploymentExample.java @@ -38,7 +38,7 @@ * Demonstrates how to explicitly deploy a task. Note that * it is very rare when you would need such functionality as tasks are * auto-deployed on demand first time you execute them. So in most cases - * you would just apply any of the {@code Ignite.execute(...)} methods directly. + * you would just transform any of the {@code Ignite.execute(...)} methods directly. * However, sometimes a task is not in local class path, so you may not even * know the code it will execute, but you still need to execute it. For example, * you have two independent components in the system, and one loads the task @@ -73,7 +73,7 @@ public static void main(String[] args) throws Exception { // onto remote nodes on demand. For this example this task is // available on the classpath, however in real life that may not // always be the case. In those cases you should use explicit - // 'IgniteCompute.localDeployTask(Class, ClassLoader) apply and + // 'IgniteCompute.localDeployTask(Class, ClassLoader) transform and // then use 'IgniteCompute.execute(String, Object)' method // passing your task name as first parameter. ignite.compute().localDeployTask(ExampleTask.class, ExampleTask.class.getClassLoader()); @@ -102,7 +102,7 @@ public static void main(String[] args) throws Exception { * Example task used to demonstrate direct task deployment through API. * For this example this task as available on the classpath, however * in real life that may not always be the case. In those cases - * you should use explicit {@link IgniteCompute#localDeployTask(Class, ClassLoader)} apply and + * you should use explicit {@link IgniteCompute#localDeployTask(Class, ClassLoader)} transform and * then use {@link IgniteCompute#execute(String, Object)} * method passing your task name as first parameter. *

diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/CacheDLCDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/CacheDLCDatasetExample.java index d77762f5df931..4c1c5f5acbce5 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/CacheDLCDatasetExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/CacheDLCDatasetExample.java @@ -21,9 +21,11 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.Ignition; -import org.apache.ignite.ml.dlc.DLCFactory; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.ml.dlc.dataset.DLCDataset; import org.apache.ignite.ml.dlc.dataset.transformer.DLCTransformers; +import org.apache.ignite.ml.dlc.impl.cache.CacheBasedDLCFactory; /** * How to create a DLC dataset from an existing Ignite Cache? @@ -43,18 +45,12 @@ public static void main(String[] args) { // context cache with specified transformation (it will be performed locally because partitions are on the // same nodes). In this case for every partition in upstream cache will be created labeled dataset partition // and this new partition will be filled with help of specified feature and label extractors. - DLCDataset dataset = DLCFactory.createDLC( - ignite, - persons, - (data, size) -> null, - DLCTransformers.upstreamToDataset((k, v) -> { - double[] row = new double[2]; - row[0] = v.age; - row[1] = v.salary; - return row; - }, 2), - DLCDataset::new - ); + DLCDataset dataset = new CacheBasedDLCFactory<>(ignite, persons) + .createDLC( + (data, size) -> null, + DLCTransformers.upstreamToDataset((k, v) -> new double[]{ v.age, v.salary }, 2), + DLCDataset::new + ); // Calculation of the mean value. This calculation will be performed in map-reduce manner. double[] mean = dataset.mean(); @@ -82,7 +78,10 @@ public static void main(String[] args) { /** */ private static IgniteCache createCache(Ignite ignite) { - IgniteCache persons = ignite.createCache("PERSONS"); + CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); + cacheConfiguration.setName("PERSONS"); + cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 2)); + IgniteCache persons = ignite.createCache(cacheConfiguration); persons.put(1, new Person("Mike", 42, 10000)); persons.put(2, new Person("John", 32, 64000)); persons.put(3, new Person("George", 53, 120000)); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/LocalDLCDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/LocalDLCDatasetExample.java index 0cb95a4e327e9..055cf73ac057f 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/LocalDLCDatasetExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/LocalDLCDatasetExample.java @@ -22,9 +22,9 @@ import java.util.Map; import org.apache.ignite.Ignite; import org.apache.ignite.Ignition; -import org.apache.ignite.ml.dlc.DLCFactory; import org.apache.ignite.ml.dlc.dataset.DLCDataset; import org.apache.ignite.ml.dlc.dataset.transformer.DLCTransformers; +import org.apache.ignite.ml.dlc.impl.local.MapBasedDLCFactory; /** * How to create a DLC dataset from an existing local data? @@ -44,18 +44,12 @@ public static void main(String[] args) { // context cache with specified transformation (it will be performed locally because partitions are on the // same nodes). In this case for every partition in upstream cache will be created labeled dataset partition // and this new partition will be filled with help of specified feature and label extractors. - DLCDataset dataset = DLCFactory.createDLC( - persons, - 2, - (data, size) -> null, - DLCTransformers.upstreamToDataset((k, v) -> { - double[] row = new double[2]; - row[0] = v.age; - row[1] = v.salary; - return row; - }, 2), - DLCDataset::new - ); + DLCDataset dataset = new MapBasedDLCFactory<>(persons, 2) + .createDLC( + (data, size) -> null, + DLCTransformers.upstreamToDataset((k, v) -> new double[]{ v.age, v.salary }, 2), + DLCDataset::new + ); // Calculation of the mean value. This calculation will be performed in map-reduce manner. double[] mean = dataset.mean(); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificDataset.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificDataset.java new file mode 100644 index 0000000000000..43f835aea4de2 --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificDataset.java @@ -0,0 +1,97 @@ +/* + * 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.examples.ml.dlc.transformer; + +import com.github.fommil.netlib.BLAS; +import java.util.Arrays; +import org.apache.ignite.ml.dlc.DLC; +import org.apache.ignite.ml.dlc.dataset.DLCWrapper; + +/** + * Algorithm-specific dataset which provides API allows to make gradient descent steps and maintain iteration number. + * + * @param type of an upstream value key + * @param type of an upstream value + */ +public class AlgorithmSpecificDataset extends DLCWrapper { + /** BlAS instance. */ + private static final BLAS blas = BLAS.getInstance(); + + /** + * Constructs a new instance of Distributed Learning Context wrapper + * + * @param delegate delegate which actually performs base functions + */ + public AlgorithmSpecificDataset( + DLC delegate) { + super(delegate); + } + + /** + * Make gradient step. + * + * @param x point to calculate gradient + * @return gradient + */ + public double[] makeGradientStep(double[] x) { + return compute((part, partIdx) -> { + AlgorithmSpecificRecoverableData recoverableData = part.getRecoverableData(); + AlgorithmSpecificReplicatedData replicatedData = part.getReplicatedData(); + + int iteration = replicatedData.getIteration(); + + double[] a = recoverableData.getA(); + double[] b = recoverableData.getB(); + int rows = recoverableData.getRows(); + int cols = recoverableData.getCols(); + double[] bCp = Arrays.copyOf(b, b.length); + + + blas.dgemv("N", rows, cols, 1.0, a, Math.max(0, rows), x, 1, -1.0, bCp, 1); + + double[] gradient = new double[cols]; + blas.dgemv("T", rows, cols, 1.0, a, Math.max(0, rows), bCp, 1, 0, gradient, 1); + + replicatedData.setIteration(iteration + 1); + + System.err.println("A = " + Arrays.toString(a) + ", B = " + Arrays.toString(b) + ", rows = " + rows + ", cols = " + cols + ", iter = " + iteration + ", g = " + Arrays.toString(gradient)); + + + return gradient; + }, this::reduce); + } + + /** + * Calculates sum of two vectors. + * + * @param a first vector + * @param b second vector + * @return sum of two vectors + */ + private double[] reduce(double[] a, double[] b) { + if (a == null) + return b; + + if (b == null) + return a; + + blas.daxpy(a.length, 1.0, a, 1, b, 1); + + return b; + } +} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificRecoverableData.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificRecoverableData.java new file mode 100644 index 0000000000000..5695d208dc073 --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificRecoverableData.java @@ -0,0 +1,73 @@ +/* + * 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.examples.ml.dlc.transformer; + +/** + * Algorithm specific recoverable data consists of feature matrix supplemented by the column containing "1" and labels. + */ +public class AlgorithmSpecificRecoverableData implements AutoCloseable { + /** Matrix supplemented by the column containing "1". */ + private final double[] a; + + /** Number of rows. */ + private final int rows; + + /** Number of columns. */ + private final int cols; + + /** Labels. */ + private final double[] b; + + /** + * Constructs a new instance of algorithm specific recoverable data. + * + * @param a matrix supplemented by the column containing "1" + * @param b labels + */ + public AlgorithmSpecificRecoverableData(double[] a, int rows, int cols, double[] b) { + this.a = a; + this.rows = rows; + this.cols = cols; + this.b = b; + } + + /** */ + public double[] getA() { + return a; + } + + /** */ + public double[] getB() { + return b; + } + + /** */ + public int getRows() { + return rows; + } + + /** */ + public int getCols() { + return cols; + } + + /** */ + @Override public void close() { + // do nothing + } +} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificRecoverableDataTransformer.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificRecoverableDataTransformer.java new file mode 100644 index 0000000000000..3182741b7a7d9 --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificRecoverableDataTransformer.java @@ -0,0 +1,45 @@ +/* + * 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.examples.ml.dlc.transformer; + +import org.apache.ignite.ml.dlc.dataset.part.DLCLabeledDatasetPartitionRecoverable; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +/** + * Transformer which transforms dataset recoverable data to algorithm specific recoverable data by supplementing column + * with value "1". + */ +public class AlgorithmSpecificRecoverableDataTransformer + implements IgniteFunction { + /** */ + private static final long serialVersionUID = 2292486552367922497L; + + /** {@inheritDoc} */ + @Override public AlgorithmSpecificRecoverableData apply(DLCLabeledDatasetPartitionRecoverable recoverable) { + int rows = recoverable.getRows(); + int cols = recoverable.getCols(); + + // Add first column with values equal to 1.0. + double[] supplementedMatrix = new double[rows * (cols + 1)]; + for (int i = 0; i < rows; i++) + supplementedMatrix[i] = 1.0; + System.arraycopy(recoverable.getFeatures(), 0, supplementedMatrix, rows, rows * cols); + + return new AlgorithmSpecificRecoverableData(supplementedMatrix, rows, cols + 1, recoverable.getLabels()); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/replicated/DLCDatasetPartitionReplicated.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificReplicatedData.java similarity index 62% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/replicated/DLCDatasetPartitionReplicated.java rename to examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificReplicatedData.java index 38623a37b4a42..9400a298febbe 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/replicated/DLCDatasetPartitionReplicated.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificReplicatedData.java @@ -15,14 +15,27 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlc.dataset.part.replicated; +package org.apache.ignite.examples.ml.dlc.transformer; import java.io.Serializable; /** - * Replicated part of DLC dataset partition. + * Algorithm-specific replicated data maintaining number of iteration. */ -public class DLCDatasetPartitionReplicated implements Serializable { +public class AlgorithmSpecificReplicatedData implements Serializable { /** */ - private static final long serialVersionUID = 3579701425183018483L; + private static final long serialVersionUID = 7593887973722933436L; + + /** Number of iteration. */ + private int iteration; + + /** */ + public int getIteration() { + return iteration; + } + + /** */ + public void setIteration(int iteration) { + this.iteration = iteration; + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/replicated/DLCLabeledDatasetPartitionReplicated.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificReplicatedDataTransformer.java similarity index 50% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/replicated/DLCLabeledDatasetPartitionReplicated.java rename to examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificReplicatedDataTransformer.java index 9cc34917365df..8ca2af818b109 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/replicated/DLCLabeledDatasetPartitionReplicated.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificReplicatedDataTransformer.java @@ -15,14 +15,24 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlc.dataset.part.replicated; +package org.apache.ignite.examples.ml.dlc.transformer; -import java.io.Serializable; +import org.apache.ignite.ml.dlc.DLCPartitionReplicatedTransformer; +import org.apache.ignite.ml.dlc.DLCUpstreamEntry; /** - * Replicated part of DLC labeled dataset partition. + * Transformer which transforms upstream data into algorithm-specific replicated data. + * + * @param type of an upstream value key + * @param type of an upstream value */ -public class DLCLabeledDatasetPartitionReplicated implements Serializable { +public class AlgorithmSpecificReplicatedDataTransformer implements DLCPartitionReplicatedTransformer { /** */ - private static final long serialVersionUID = 132087952507878026L; + private static final long serialVersionUID = -3720585607267757357L; + + /** {@inheritDoc} */ + @Override public AlgorithmSpecificReplicatedData transform(Iterable> upstreamData, + Long upstreamDataSize) { + return new AlgorithmSpecificReplicatedData(); + } } diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/DLCTransformerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/DLCTransformerExample.java new file mode 100644 index 0000000000000..11c41b6be0ec5 --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/DLCTransformerExample.java @@ -0,0 +1,108 @@ +/* + * 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.examples.ml.dlc.transformer; + +import com.github.fommil.netlib.BLAS; +import java.util.Arrays; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.ml.dlc.dataset.transformer.DLCTransformers; +import org.apache.ignite.ml.dlc.impl.cache.CacheBasedDLCFactory; + +/** How to transform DLC into algorithm-specific view? */ +public class DLCTransformerExample { + /** Run example. */ + public static void main(String[] args) { + BLAS bas = BLAS.getInstance(); + + + + try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { + System.out.println(">>> D-Learn Cache Dataset example started."); + + IgniteCache persons = createCache(ignite); + + // Initialization of the d-learn context. After this step context cache will be created with partitions + // placed on the same nodes as the upstream Ignite Cache. + + // Loading of the d-learn context. During this step data will be transferred from the upstream cache to + // context cache with specified transformation (it will be performed locally because partitions are on the + // same nodes). In this case for every partition in upstream cache will be created labeled dataset partition + // and this new partition will be filled with help of specified feature and label extractors. + + AlgorithmSpecificDataset dataset = new CacheBasedDLCFactory<>(ignite, persons) + .createDLC( + new AlgorithmSpecificReplicatedDataTransformer<>(), + DLCTransformers.upstreamToLabeledDataset( + (k, v) -> new double[]{ v.age }, + (k, v) -> v.salary, + 1 + ).andThen(new AlgorithmSpecificRecoverableDataTransformer()), + AlgorithmSpecificDataset::new + ); + + double[] x = new double[2]; + + for (int iter = 0; iter < 100; iter++) { + double[] gradient = dataset.makeGradientStep(x); + System.out.println("Point : " + Arrays.toString(x) + ", gradient : " + Arrays.toString(gradient)); + for (int i = 0; i < x.length; i++) + x[i] -= 0.01 * gradient[i]; + } + + System.out.println("New point : " + Arrays.toString(x)); + + System.out.println(">>> D-Learn Cache Dataset example completed."); + } + } + + /** */ + private static IgniteCache createCache(Ignite ignite) { + CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); + cacheConfiguration.setName("PERSONS"); + cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 1)); + IgniteCache persons = ignite.createCache(cacheConfiguration); + persons.put(1, new Person("Mike", 10, 100)); + persons.put(2, new Person("John", 20, 200)); + persons.put(3, new Person("George", 30, 300)); + persons.put(4, new Person("Karl", 40, 400)); + return persons; + } + + /** */ + private static class Person { + /** */ + private final String name; + + /** */ + private final double age; + + /** */ + private final double salary; + + /** */ + public Person(String name, double age, double salary) { + this.name = name; + this.age = age; + this.salary = salary; + } + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCFactory.java index 95657ccb6f307..a737b551e42a8 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCFactory.java @@ -18,139 +18,45 @@ package org.apache.ignite.ml.dlc; import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import javax.cache.Cache; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.Ignition; -import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.cache.affinity.AffinityFunction; -import org.apache.ignite.cache.query.QueryCursor; -import org.apache.ignite.cache.query.ScanQuery; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.ml.dlc.impl.cache.CacheBasedDLCImpl; -import org.apache.ignite.ml.dlc.impl.cache.DLCAffinityFunctionWrapper; -import org.apache.ignite.ml.dlc.impl.cache.util.DLCUpstreamCursorAdapter; -import org.apache.ignite.ml.dlc.impl.local.MapBasedDLCImpl; -import org.apache.ignite.ml.dlc.impl.local.util.DLCUpstreamMapAdapter; import org.apache.ignite.ml.math.functions.IgniteFunction; /** - * Distributed Learning Context factory which produces contexts based on Ignite Cache and local Map. + * DLC factory which produces distributed learning contexts. + * + * @param type of an upstream value key + * @param type of an upstream value */ -public class DLCFactory { +public interface DLCFactory { /** - * Constructs a new instance of Distributed Learning Context based on the specified upstream Ignite Cache and uses - * Ignite Cache as reliable storage. + * Constructs a new instance of distributed learning context using specified replicated data transformer, + * recoverable data transformer and DLC wrapper. * - * @param ignite Ignite instance - * @param upstreamCache upstream cache * @param replicatedTransformer replicated data transformer * @param recoverableTransformer recoverable data transformer - * @param wrapDLC learning context wrapper - * @param type of an upstream value key - * @param type of an upstream value + * @param wrapDLC DLC wrapper * @param type of replicated data of a partition * @param type of recoverable data of a partition * @param type of returned learning context * @return distributed learning context */ - @SuppressWarnings("unchecked") - public static > I createDLC( - Ignite ignite, - IgniteCache upstreamCache, + public > I createDLC( DLCPartitionReplicatedTransformer replicatedTransformer, DLCPartitionRecoverableTransformer recoverableTransformer, - IgniteFunction, I> wrapDLC) { - UUID dlcId = UUID.randomUUID(); - - AffinityFunction upstreamCacheAffinity = upstreamCache.getConfiguration(CacheConfiguration.class).getAffinity(); - - CacheConfiguration> dlcCacheCfg = new CacheConfiguration<>(); - dlcCacheCfg.setName(dlcId.toString()); - dlcCacheCfg.setAffinity(new DLCAffinityFunctionWrapper(upstreamCacheAffinity)); - - IgniteCache> dlcCache = ignite.createCache(dlcCacheCfg); - - Affinity affinity = ignite.affinity(dlcCache.getName()); - int partitions = affinity.partitions(); - - for (int partIdx = 0; partIdx < partitions; partIdx++) { - int currPartIdx = partIdx; - - ignite.compute().affinityRun(Collections.singletonList(dlcCache.getName()), partIdx, () -> { - Ignite locIgnite = Ignition.localIgnite(); - IgniteCache locUpstreamCache = locIgnite.cache(upstreamCache.getName()); - - ScanQuery qry = new ScanQuery<>(); - qry.setLocal(true); - qry.setPartition(currPartIdx); - - long cnt = locUpstreamCache.localSizeLong(currPartIdx); - Q replicated; - try (QueryCursor> cursor = locUpstreamCache.query(qry)) { - replicated = replicatedTransformer.apply(new DLCUpstreamCursorAdapter<>(cursor), cnt); - } - DLCPartition part = new DLCPartition<>(replicated, recoverableTransformer); - dlcCache.put(currPartIdx, part); - }); - } - - DLC dlc = new CacheBasedDLCImpl<>(ignite, upstreamCache, dlcCache, dlcId); - - return wrapDLC.apply(dlc); - } + IgniteFunction, I> wrapDLC); /** - * Constructs a new instance of Distributed Learning Context based on the specified Map and uses local HashMap as - * reliable storage. + * Constructs a new instance of distributed learning context using specified replicated data transformer and + * replicated data transformer. * - * @param upstreamData upstream data - * @param partitions number of partitions * @param replicatedTransformer replicated data transformer * @param recoverableTransformer recoverable data transformer - * @param wrapDLC learning context wrapper - * @param type of an upstream value key - * @param type of an upstream value - * @param type of replicated data of a partition + * @param type of an upstream value * @param type of recoverable data of a partition - * @param type of returned learning context - * @return distributed learning context + * @return distributed learning context */ - public static > I createDLC( - Map upstreamData, int partitions, + default public DLC cteateDLC( DLCPartitionReplicatedTransformer replicatedTransformer, - DLCPartitionRecoverableTransformer recoverableTransformer, - IgniteFunction, I> wrapDLC) { - Map> dlcMap = new HashMap<>(); - - int partSize = upstreamData.size() / partitions; - - List keys = new ArrayList<>(upstreamData.keySet()); - - for (int partIdx = 0; partIdx < partitions; partIdx++) { - List partKeys = keys.subList(partIdx * partSize, Math.min((partIdx + 1) * partSize, upstreamData.size())); - Q replicated = replicatedTransformer.apply( - new DLCUpstreamMapAdapter<>(upstreamData, partKeys), - (long) partKeys.size() - ); - W recoverable = recoverableTransformer.apply( - new DLCUpstreamMapAdapter<>(upstreamData, partKeys), - (long) partKeys.size(), - replicated - ); - DLCPartition part = new DLCPartition<>(replicated, null); - part.setRecoverableData(recoverable); - dlcMap.put(partIdx, part); - } - - DLC dlc = new MapBasedDLCImpl<>(dlcMap, partitions); - - return wrapDLC.apply(dlc); + DLCPartitionRecoverableTransformer recoverableTransformer) { + return createDLC(replicatedTransformer, recoverableTransformer, e -> e); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartition.java index d7630aa9968a7..4b1763639ae43 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartition.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartition.java @@ -49,7 +49,8 @@ public class DLCPartition * @param replicatedData replicated data * @param recoverableDataTransformer transformer of the recoverable part of this partition */ - public DLCPartition(Q replicatedData, DLCPartitionRecoverableTransformer recoverableDataTransformer) { + public DLCPartition(Q replicatedData, + DLCPartitionRecoverableTransformer recoverableDataTransformer) { this.replicatedData = replicatedData; this.recoverableDataTransformer = recoverableDataTransformer; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableTransformer.java index 94c4a03566e97..1837f86f4f7de 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableTransformer.java @@ -17,25 +17,39 @@ package org.apache.ignite.ml.dlc; -import org.apache.ignite.ml.math.functions.IgniteTriFunction; +import java.io.Serializable; +import org.apache.ignite.ml.math.functions.IgniteFunction; /** + * Transformer of the partition recoverable data. * - * - * @param - * @param - * @param - * @param + * @param type of an upstream value key + * @param type of an upstream value + * @param type of replicated data of a partition + * @param type of recoverable data of a partition */ @FunctionalInterface -public interface DLCPartitionRecoverableTransformer - extends IgniteTriFunction>, Long, Q, W> { +public interface DLCPartitionRecoverableTransformer + extends Serializable { + /** + * Transforms upstream data to the partition recoverable data. + * + * @param upstreamData upstream data + * @param upstreamDataSize upstream data size + * @param replicatedData replicated data + * @return recoverable data + */ + public W transform(Iterable> upstreamData, Long upstreamDataSize, Q replicatedData); + /** + * Makes a composition of functions. * - * @param upstreamData - * @param upstreamDataSize - * @param replicatedData - * @return + * @param after function will be called after this one + * @param type of recoverable data of a partition + * @return new transformer */ - W apply(Iterable> upstreamData, Long upstreamDataSize, Q replicatedData); + default DLCPartitionRecoverableTransformer andThen( + IgniteFunction after) { + return (upData, upDataSize, replicatedData) -> after.apply(transform(upData, upDataSize, replicatedData)); + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedTransformer.java index 241a41d61ff36..0d01838f2fb04 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedTransformer.java @@ -17,24 +17,35 @@ package org.apache.ignite.ml.dlc; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import java.io.Serializable; +import org.apache.ignite.ml.math.functions.IgniteFunction; /** + * Transformer of the partition replicated data. * - * - * @param - * @param - * @param + * @param type of an upstream value key + * @param type of an upstream value + * @param type of replicated data of a partition */ @FunctionalInterface -public interface DLCPartitionReplicatedTransformer - extends IgniteBiFunction>, Long, Q> { +public interface DLCPartitionReplicatedTransformer extends Serializable { /** + * Transforms upstream data to the partition replicated data. * + * @param upstreamData upstream data + * @param upstreamDataSize upstream data size + * @return replicated data + */ + public Q transform(Iterable> upstreamData, Long upstreamDataSize); + + /** + * Makes a composition of functions. * - * @param upstreamData - * @param upstreamDataSize - * @return + * @param after function will be called after this one + * @param type of replicated data of a partition + * @return new transformer */ - Q apply(Iterable> upstreamData, Long upstreamDataSize); + default DLCPartitionReplicatedTransformer andThen(IgniteFunction after) { + return (upData, upDataSize) -> after.apply(transform(upData, upDataSize)); + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCDataset.java index c8ecec6d9ba20..6347db386964f 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCDataset.java @@ -18,9 +18,9 @@ package org.apache.ignite.ml.dlc.dataset; import com.github.fommil.netlib.BLAS; +import java.io.Serializable; import org.apache.ignite.ml.dlc.DLC; -import org.apache.ignite.ml.dlc.dataset.part.recoverable.DLCDatasetPartitionRecoverable; -import org.apache.ignite.ml.dlc.dataset.part.replicated.DLCDatasetPartitionReplicated; +import org.apache.ignite.ml.dlc.dataset.part.DLCDatasetPartitionRecoverable; /** * Dataset provides API to work with dataset. @@ -28,7 +28,7 @@ * @param type of an upstream value key * @param type of an upstream value */ -public class DLCDataset extends DLCWrapper { +public class DLCDataset extends DLCWrapper { /** BLAS (Basic Linear Algebra Subprograms) instance. */ private static final BLAS blas = BLAS.getInstance(); /** @@ -36,8 +36,7 @@ public class DLCDataset extends DLCWrapper delegate) { + public DLCDataset(DLC delegate) { super(delegate); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCLabeledDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCLabeledDataset.java index 3a44203b0d389..ed282c06d5d5a 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCLabeledDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCLabeledDataset.java @@ -17,9 +17,9 @@ package org.apache.ignite.ml.dlc.dataset; +import java.io.Serializable; import org.apache.ignite.ml.dlc.DLC; -import org.apache.ignite.ml.dlc.dataset.part.recoverable.DLCLabeledDatasetPartitionRecoverable; -import org.apache.ignite.ml.dlc.dataset.part.replicated.DLCLabeledDatasetPartitionReplicated; +import org.apache.ignite.ml.dlc.dataset.part.DLCLabeledDatasetPartitionRecoverable; /** * Dataset provides API to work with labeled dataset. @@ -27,15 +27,15 @@ * @param type of an upstream value key * @param type of an upstream value */ -public class DLCLabeledDataset - extends DLCWrapper { +public class DLCLabeledDataset + extends DLCWrapper { /** * Constructs a new instance of Distributed Learning Context wrapper * * @param delegate delegate which actually performs base functions */ public DLCLabeledDataset( - DLC delegate) { + DLC delegate) { super(delegate); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCDatasetPartitionRecoverable.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/DLCDatasetPartitionRecoverable.java similarity index 97% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCDatasetPartitionRecoverable.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/DLCDatasetPartitionRecoverable.java index 6fdc5b82b1259..15969b00dac69 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCDatasetPartitionRecoverable.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/DLCDatasetPartitionRecoverable.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlc.dataset.part.recoverable; +package org.apache.ignite.ml.dlc.dataset.part; /** * Recoverable part of a DLC dataset partition contains matrix with features in a dense flat column-major format. diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCLabeledDatasetPartitionRecoverable.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/DLCLabeledDatasetPartitionRecoverable.java similarity index 97% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCLabeledDatasetPartitionRecoverable.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/DLCLabeledDatasetPartitionRecoverable.java index 1c85df865c361..1eaade1856504 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/recoverable/DLCLabeledDatasetPartitionRecoverable.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/DLCLabeledDatasetPartitionRecoverable.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlc.dataset.part.recoverable; +package org.apache.ignite.ml.dlc.dataset.part; /** * Recoverable part of a DLC labeled dataset partition contains matrix of features in a dense flat column-major format diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/DLCTransformers.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/DLCTransformers.java index fa17c9315519f..5ae73252b39d1 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/DLCTransformers.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/DLCTransformers.java @@ -17,10 +17,9 @@ package org.apache.ignite.ml.dlc.dataset.transformer; +import java.io.Serializable; import org.apache.ignite.ml.dlc.dataset.DLCDataset; import org.apache.ignite.ml.dlc.dataset.DLCLabeledDataset; -import org.apache.ignite.ml.dlc.dataset.transformer.recoverable.UpstreamToDatasetTransformer; -import org.apache.ignite.ml.dlc.dataset.transformer.recoverable.UpstreamToLabeledDatasetTransformer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; /** @@ -36,9 +35,10 @@ public class DLCTransformers { * @param features number of features * @param type of an upstream value key * @param type of an upstream value + * @param type of replicated data of a partition * @return transformer */ - public static UpstreamToDatasetTransformer upstreamToDataset( + public static UpstreamToDatasetTransformer upstreamToDataset( IgniteBiFunction featureExtractor, int features) { return new UpstreamToDatasetTransformer<>(featureExtractor, features); } @@ -54,7 +54,7 @@ public static UpstreamToDatasetTransformer upstreamToDataset( * @param type of an upstream value * @return transformer */ - public static UpstreamToLabeledDatasetTransformer upstreamToLabeledDataset( + public static UpstreamToLabeledDatasetTransformer upstreamToLabeledDataset( IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor, int features) { return new UpstreamToLabeledDatasetTransformer<>(featureExtractor, lbExtractor, features); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/recoverable/UpstreamToDatasetTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToDatasetTransformer.java similarity index 81% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/recoverable/UpstreamToDatasetTransformer.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToDatasetTransformer.java index 46f872afe0f79..cb441b6cd8b6c 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/recoverable/UpstreamToDatasetTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToDatasetTransformer.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlc.dataset.transformer.recoverable; +package org.apache.ignite.ml.dlc.dataset.transformer; +import java.io.Serializable; import org.apache.ignite.ml.dlc.DLCPartitionRecoverableTransformer; import org.apache.ignite.ml.dlc.DLCUpstreamEntry; import org.apache.ignite.ml.dlc.dataset.DLCDataset; -import org.apache.ignite.ml.dlc.dataset.part.recoverable.DLCDatasetPartitionRecoverable; -import org.apache.ignite.ml.dlc.dataset.part.replicated.DLCDatasetPartitionReplicated; +import org.apache.ignite.ml.dlc.dataset.part.DLCDatasetPartitionRecoverable; import org.apache.ignite.ml.math.functions.IgniteBiFunction; /** @@ -30,8 +30,8 @@ * @param type of an upstream value key * @param type of an upstream value */ -public class UpstreamToDatasetTransformer - implements DLCPartitionRecoverableTransformer { +public class UpstreamToDatasetTransformer + implements DLCPartitionRecoverableTransformer { /** */ private static final long serialVersionUID = -3713681392540367983L; @@ -61,8 +61,8 @@ public UpstreamToDatasetTransformer( * @param replicatedData replicated data * @return dataset partition recoverable data */ - @Override public DLCDatasetPartitionRecoverable apply(Iterable> upstreamData, Long upstreamDataSize, - DLCDatasetPartitionReplicated replicatedData) { + @Override public DLCDatasetPartitionRecoverable transform(Iterable> upstreamData, Long upstreamDataSize, + Q replicatedData) { int rows = Math.toIntExact(upstreamDataSize), cols = features; double[] features = new double[rows * cols]; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/recoverable/UpstreamToLabeledDatasetTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToLabeledDatasetTransformer.java similarity index 82% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/recoverable/UpstreamToLabeledDatasetTransformer.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToLabeledDatasetTransformer.java index 12345ac256fd9..e80d448a07f51 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/recoverable/UpstreamToLabeledDatasetTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToLabeledDatasetTransformer.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlc.dataset.transformer.recoverable; +package org.apache.ignite.ml.dlc.dataset.transformer; +import java.io.Serializable; import org.apache.ignite.ml.dlc.DLCPartitionRecoverableTransformer; import org.apache.ignite.ml.dlc.DLCUpstreamEntry; import org.apache.ignite.ml.dlc.dataset.DLCLabeledDataset; -import org.apache.ignite.ml.dlc.dataset.part.recoverable.DLCLabeledDatasetPartitionRecoverable; -import org.apache.ignite.ml.dlc.dataset.part.replicated.DLCLabeledDatasetPartitionReplicated; +import org.apache.ignite.ml.dlc.dataset.part.DLCLabeledDatasetPartitionRecoverable; import org.apache.ignite.ml.math.functions.IgniteBiFunction; /** @@ -30,9 +30,8 @@ * @param type of an upstream value key * @param type of an upstream value */ -public class UpstreamToLabeledDatasetTransformer - implements DLCPartitionRecoverableTransformer { +public class UpstreamToLabeledDatasetTransformer + implements DLCPartitionRecoverableTransformer { /** */ private static final long serialVersionUID = -1224768715207401297L; @@ -68,8 +67,8 @@ public UpstreamToLabeledDatasetTransformer( * @param replicatedData replicated data * @return labeled dataset recoverable data */ - @Override public DLCLabeledDatasetPartitionRecoverable apply(Iterable> upstreamData, - Long upstreamDataSize, DLCLabeledDatasetPartitionReplicated replicatedData) { + @Override public DLCLabeledDatasetPartitionRecoverable transform(Iterable> upstreamData, + Long upstreamDataSize, Q replicatedData) { int rows = Math.toIntExact(upstreamDataSize), cols = features; double[] features = new double[rows * cols]; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/CacheBasedDLCFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/CacheBasedDLCFactory.java new file mode 100644 index 0000000000000..1630bde4f5a8c --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/CacheBasedDLCFactory.java @@ -0,0 +1,109 @@ +/* + * 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.ml.dlc.impl.cache; + +import java.io.Serializable; +import java.util.Collections; +import java.util.UUID; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cache.affinity.AffinityFunction; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.ml.dlc.DLC; +import org.apache.ignite.ml.dlc.DLCFactory; +import org.apache.ignite.ml.dlc.DLCPartition; +import org.apache.ignite.ml.dlc.DLCPartitionRecoverableTransformer; +import org.apache.ignite.ml.dlc.DLCPartitionReplicatedTransformer; +import org.apache.ignite.ml.dlc.impl.cache.util.DLCAffinityFunctionWrapper; +import org.apache.ignite.ml.dlc.impl.cache.util.DLCUpstreamCursorAdapter; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +/** + * DLC factory which produces distributed learning contexts based on the Ignite Cache. + * + * @param type of an upstream value key + * @param type of an upstream value + */ +public class CacheBasedDLCFactory implements DLCFactory { + /** Ignite instance. */ + private final Ignite ignite; + + /** Upstream cache. */ + private final IgniteCache upstreamCache; + + /** + * Constructs a new instance of cache based DLC factory. + * + * @param ignite Ignite instance + * @param upstreamCache upstream cache + */ + public CacheBasedDLCFactory(Ignite ignite, IgniteCache upstreamCache) { + this.ignite = ignite; + this.upstreamCache = upstreamCache; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public > I createDLC( + DLCPartitionReplicatedTransformer replicatedTransformer, + DLCPartitionRecoverableTransformer recoverableTransformer, + IgniteFunction, I> wrapDLC) { + UUID dlcId = UUID.randomUUID(); + + AffinityFunction upstreamCacheAffinity = upstreamCache.getConfiguration(CacheConfiguration.class).getAffinity(); + + CacheConfiguration> dlcCacheCfg = new CacheConfiguration<>(); + dlcCacheCfg.setName(dlcId.toString()); + dlcCacheCfg.setAffinity(new DLCAffinityFunctionWrapper(upstreamCacheAffinity)); + + IgniteCache> dlcCache = ignite.createCache(dlcCacheCfg); + + Affinity affinity = ignite.affinity(dlcCache.getName()); + int partitions = affinity.partitions(); + + for (int partIdx = 0; partIdx < partitions; partIdx++) { + int currPartIdx = partIdx; + + ignite.compute().affinityRun(Collections.singletonList(dlcCache.getName()), partIdx, () -> { + Ignite locIgnite = Ignition.localIgnite(); + IgniteCache locUpstreamCache = locIgnite.cache(upstreamCache.getName()); + + ScanQuery qry = new ScanQuery<>(); + qry.setLocal(true); + qry.setPartition(currPartIdx); + + long cnt = locUpstreamCache.localSizeLong(currPartIdx); + Q replicated; + try (QueryCursor> cursor = locUpstreamCache.query(qry)) { + replicated = replicatedTransformer.transform(new DLCUpstreamCursorAdapter<>(cursor), cnt); + } + DLCPartition part = new DLCPartition<>(replicated, recoverableTransformer); + dlcCache.put(currPartIdx, part); + }); + } + + DLC dlc = new CacheBasedDLCImpl<>(ignite, upstreamCache, dlcCache, dlcId); + + return wrapDLC.apply(dlc); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/CacheBasedDLCPartitionBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/CacheBasedDLCPartitionBuilder.java index 3cc40a774471d..3b31859ca0ab4 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/CacheBasedDLCPartitionBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/CacheBasedDLCPartitionBuilder.java @@ -35,8 +35,7 @@ import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.ml.dlc.DLCPartition; -import org.apache.ignite.ml.dlc.DLCUpstreamEntry; -import org.apache.ignite.ml.math.functions.IgniteTriFunction; +import org.apache.ignite.ml.dlc.DLCPartitionRecoverableTransformer; /** * Distributed Learning Context partition builder which constructs a partition from two parts: replicated data which is @@ -142,7 +141,7 @@ public DLCPartition build() { * @return recoverable data */ private W loadRecoverableData( - IgniteTriFunction>, Long, Q, W> recoverableDataTransformer, + DLCPartitionRecoverableTransformer recoverableDataTransformer, Q replicatedData) { Affinity upstreamCacheAffinity = ignite.affinity(upstreamCacheName); ClusterNode partNode = upstreamCacheAffinity.mapPartitionToNode(partIdx); @@ -162,7 +161,7 @@ private W loadRecoverableData( // TODO: how to guarantee that cache size will not be changed between these calls? long cnt = locUpstreamCache.localSizeLong(partIdx); try (QueryCursor> cursor = locUpstreamCache.query(qry)) { - return recoverableDataTransformer.apply(new DLCUpstreamCursorAdapter<>(cursor), cnt, replicatedData); + return recoverableDataTransformer.transform(new DLCUpstreamCursorAdapter<>(cursor), cnt, replicatedData); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/DLCAffinityFunctionWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCAffinityFunctionWrapper.java similarity index 98% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/DLCAffinityFunctionWrapper.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCAffinityFunctionWrapper.java index 261f1357600df..2ade520e9e703 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/DLCAffinityFunctionWrapper.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCAffinityFunctionWrapper.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlc.impl.cache; +package org.apache.ignite.ml.dlc.impl.cache.util; import java.util.List; import java.util.UUID; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCFactory.java new file mode 100644 index 0000000000000..2d2c54370c414 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCFactory.java @@ -0,0 +1,88 @@ +/* + * 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.ml.dlc.impl.local; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.ignite.ml.dlc.DLC; +import org.apache.ignite.ml.dlc.DLCFactory; +import org.apache.ignite.ml.dlc.DLCPartition; +import org.apache.ignite.ml.dlc.DLCPartitionRecoverableTransformer; +import org.apache.ignite.ml.dlc.DLCPartitionReplicatedTransformer; +import org.apache.ignite.ml.dlc.impl.local.util.DLCUpstreamMapAdapter; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +/** + * DLC factory which produces distributed learning contexts based on the local {@code Map}. + * + * @param type of an upstream value key + * @param type of an upstream value + */ +public class MapBasedDLCFactory implements DLCFactory { + /** Upstream data. */ + private final Map upstreamData; + + /** Number of partitions. */ + private final int partitions; + + /** + * Constructs a new instance of {@code Map} based DLC factory. + * + * @param upstreamData upstream data + * @param partitions partitions + */ + public MapBasedDLCFactory(Map upstreamData, int partitions) { + this.upstreamData = upstreamData; + this.partitions = partitions; + } + + /** {@inheritDoc} */ + @Override public > I createDLC( + DLCPartitionReplicatedTransformer replicatedTransformer, + DLCPartitionRecoverableTransformer recoverableTransformer, + IgniteFunction, I> wrapDLC) { + Map> dlcMap = new HashMap<>(); + + int partSize = upstreamData.size() / partitions; + + List keys = new ArrayList<>(upstreamData.keySet()); + + for (int partIdx = 0; partIdx < partitions; partIdx++) { + List partKeys = keys.subList(partIdx * partSize, Math.min((partIdx + 1) * partSize, upstreamData.size())); + Q replicated = replicatedTransformer.transform( + new DLCUpstreamMapAdapter<>(upstreamData, partKeys), + (long) partKeys.size() + ); + W recoverable = recoverableTransformer.transform( + new DLCUpstreamMapAdapter<>(upstreamData, partKeys), + (long) partKeys.size(), + replicated + ); + DLCPartition part = new DLCPartition<>(replicated, null); + part.setRecoverableData(recoverable); + dlcMap.put(partIdx, part); + } + + DLC dlc = new MapBasedDLCImpl<>(dlcMap, partitions); + + return wrapDLC.apply(dlc); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java b/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java index c7f373f7fc77f..f6cf2d556f1fe 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java @@ -6,9 +6,9 @@ import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.dlc.DLCFactory; import org.apache.ignite.ml.dlc.dataset.DLCDataset; import org.apache.ignite.ml.dlc.dataset.transformer.DLCTransformers; +import org.apache.ignite.ml.dlc.impl.cache.CacheBasedDLCFactory; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; /** */ @@ -49,13 +49,12 @@ public void testTest() { for (int i = 0; i < 40; i++) cache.put(i, i); - DLCDataset dataset = DLCFactory.createDLC( - ignite, - cache, - (data, size) -> null, - DLCTransformers.upstreamToDataset((k, v) -> new double[]{1, 2, 3}, 3), - DLCDataset::new - ); + DLCDataset dataset = new CacheBasedDLCFactory<>(ignite, cache) + .createDLC( + (data, size) -> null, + DLCTransformers.upstreamToDataset((k, v) -> new double[]{1, 2, 3}, 3), + DLCDataset::new + ); // Calculation of the mean value. This calculation will be performed in map-reduce manner. double[] mean = dataset.mean(); From 9d72b1cb7b8129fe3290fe6ab2bd8433f90f9aae Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Fri, 26 Jan 2018 20:14:39 +0300 Subject: [PATCH 18/38] IGNITE-7437 Update DLC and add examples. --- .../CacheBasedDatasetExample.java} | 21 +- .../LocalBasedDatasetExample.java} | 22 +- .../ml/{dlc => dataset}/package-info.java | 2 +- .../transformer/AlgorithmSpecificDataset.java | 97 ------- .../AlgorithmSpecificRecoverableData.java | 73 ------ ...thmSpecificRecoverableDataTransformer.java | 45 ---- .../transformer/DLCTransformerExample.java | 108 -------- .../org/apache/ignite/ml/dataset/Dataset.java | 210 +++++++++++++++ .../ignite/ml/dataset/DatasetBuilder.java | 28 +- .../ignite/ml/dataset/DatasetFactory.java | 238 +++++++++++++++++ .../PartitionContextBuilder.java} | 28 +- .../ml/dataset/PartitionDataBuilder.java | 51 ++++ .../PartitionUpstreamEntry.java} | 6 +- .../ignite/ml/dataset/api/DatasetWrapper.java | 46 ++++ .../api/SimpleDataset.java} | 50 ++-- .../ml/dataset/api/SimpleLabeledDataset.java | 23 +- .../builder/context/EmptyContextBuilder.java | 30 +++ .../data/SimpleDatasetDataBuilder.java | 59 +++++ .../data/SimpleLabeledDatasetDataBuilder.java | 66 +++++ .../ml/dataset/api/context/EmptyContext.java | 23 ++ .../api/data/SimpleDatasetData.java} | 15 +- .../api/data/SimpleLabeledDatasetData.java} | 29 +-- .../dataset/impl/cache/CacheBasedDataset.java | 146 +++++++++++ .../impl/cache/CacheBasedDatasetBuilder.java | 90 +++++++ .../dataset/impl/cache/util/ComputeUtils.java | 243 ++++++++++++++++++ .../util/DatasetAffinityFunctionWrapper.java} | 6 +- .../cache/util/PartitionDataStorage.java} | 43 ++-- .../util/PartitionNotFoundException.java} | 6 +- .../cache/util/UpstreamCursorAdapter.java | 58 +++++ .../ml/dataset/impl/local/LocalDataset.java | 59 +++++ .../impl/local/LocalDatasetBuilder.java | 107 ++++++++ .../java/org/apache/ignite/ml/dlc/DLC.java | 122 --------- .../org/apache/ignite/ml/dlc/DLCFactory.java | 62 ----- .../apache/ignite/ml/dlc/DLCPartition.java | 77 ------ .../DLCPartitionRecoverableTransformer.java | 55 ---- .../DLCPartitionReplicatedTransformer.java | 51 ---- .../ignite/ml/dlc/dataset/DLCWrapper.java | 58 ----- .../dataset/transformer/DLCTransformers.java | 62 ----- .../UpstreamToDatasetTransformer.java | 84 ------ .../UpstreamToLabeledDatasetTransformer.java | 93 ------- .../dlc/impl/cache/CacheBasedDLCFactory.java | 109 -------- .../ml/dlc/impl/cache/CacheBasedDLCImpl.java | 158 ------------ .../util/CacheBasedDLCPartitionBuilder.java | 183 ------------- .../cache/util/DLCUpstreamCursorAdapter.java | 81 ------ .../ml/dlc/impl/local/MapBasedDLCFactory.java | 88 ------- .../ml/dlc/impl/local/MapBasedDLCImpl.java | 117 --------- .../local/util/DLCUpstreamMapAdapter.java | 85 ------ .../ColumnDecisionTreeTrainer.java | 2 +- .../org/apache/ignite/ml/DLCPlayground.java | 12 +- 49 files changed, 1555 insertions(+), 1972 deletions(-) rename examples/src/main/java/org/apache/ignite/examples/ml/{dlc/CacheDLCDatasetExample.java => dataset/CacheBasedDatasetExample.java} (87%) rename examples/src/main/java/org/apache/ignite/examples/ml/{dlc/LocalDLCDatasetExample.java => dataset/LocalBasedDatasetExample.java} (87%) rename examples/src/main/java/org/apache/ignite/examples/ml/{dlc => dataset}/package-info.java (95%) delete mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificDataset.java delete mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificRecoverableData.java delete mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificRecoverableDataTransformer.java delete mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/DLCTransformerExample.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/Dataset.java rename examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificReplicatedData.java => modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java (62%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetFactory.java rename modules/ml/src/main/java/org/apache/ignite/ml/{dlc/dataset/DLCLabeledDataset.java => dataset/PartitionContextBuilder.java} (55%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java rename modules/ml/src/main/java/org/apache/ignite/ml/{dlc/DLCUpstreamEntry.java => dataset/PartitionUpstreamEntry.java} (91%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/DatasetWrapper.java rename modules/ml/src/main/java/org/apache/ignite/ml/{dlc/dataset/DLCDataset.java => dataset/api/SimpleDataset.java} (75%) rename examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificReplicatedDataTransformer.java => modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleLabeledDataset.java (50%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/EmptyContextBuilder.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleDatasetDataBuilder.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleLabeledDatasetDataBuilder.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/EmptyContext.java rename modules/ml/src/main/java/org/apache/ignite/ml/{dlc/dataset/part/DLCDatasetPartitionRecoverable.java => dataset/api/data/SimpleDatasetData.java} (72%) rename modules/ml/src/main/java/org/apache/ignite/ml/{dlc/dataset/part/DLCLabeledDatasetPartitionRecoverable.java => dataset/api/data/SimpleLabeledDatasetData.java} (63%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java rename modules/ml/src/main/java/org/apache/ignite/ml/{dlc/impl/cache/util/DLCAffinityFunctionWrapper.java => dataset/impl/cache/util/DatasetAffinityFunctionWrapper.java} (92%) rename modules/ml/src/main/java/org/apache/ignite/ml/{dlc/impl/cache/util/DLCPartitionRecoverableDataStorage.java => dataset/impl/cache/util/PartitionDataStorage.java} (57%) rename modules/ml/src/main/java/org/apache/ignite/ml/{dlc/impl/cache/util/UpstreamPartitionNotFoundException.java => dataset/impl/cache/util/PartitionNotFoundException.java} (87%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLC.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCFactory.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartition.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableTransformer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedTransformer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCWrapper.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/DLCTransformers.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToDatasetTransformer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToLabeledDatasetTransformer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/CacheBasedDLCFactory.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/CacheBasedDLCImpl.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/CacheBasedDLCPartitionBuilder.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCUpstreamCursorAdapter.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCFactory.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCImpl.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/util/DLCUpstreamMapAdapter.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/CacheDLCDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/CacheBasedDatasetExample.java similarity index 87% rename from examples/src/main/java/org/apache/ignite/examples/ml/dlc/CacheDLCDatasetExample.java rename to examples/src/main/java/org/apache/ignite/examples/ml/dataset/CacheBasedDatasetExample.java index 4c1c5f5acbce5..fafead51bdb9c 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/CacheDLCDatasetExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/CacheBasedDatasetExample.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.examples.ml.dlc; +package org.apache.ignite.examples.ml.dataset; import java.util.Arrays; import org.apache.ignite.Ignite; @@ -23,14 +23,13 @@ import org.apache.ignite.Ignition; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.ml.dlc.dataset.DLCDataset; -import org.apache.ignite.ml.dlc.dataset.transformer.DLCTransformers; -import org.apache.ignite.ml.dlc.impl.cache.CacheBasedDLCFactory; +import org.apache.ignite.ml.dataset.DatasetFactory; +import org.apache.ignite.ml.dataset.api.SimpleDataset; /** * How to create a DLC dataset from an existing Ignite Cache? */ -public class CacheDLCDatasetExample { +public class CacheBasedDatasetExample { /** Run example. */ public static void main(String[] args) { try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { @@ -45,12 +44,12 @@ public static void main(String[] args) { // context cache with specified transformation (it will be performed locally because partitions are on the // same nodes). In this case for every partition in upstream cache will be created labeled dataset partition // and this new partition will be filled with help of specified feature and label extractors. - DLCDataset dataset = new CacheBasedDLCFactory<>(ignite, persons) - .createDLC( - (data, size) -> null, - DLCTransformers.upstreamToDataset((k, v) -> new double[]{ v.age, v.salary }, 2), - DLCDataset::new - ); + SimpleDataset dataset = DatasetFactory.createSimpleDataset( + ignite, + persons, + (k, v) -> new double[]{ v.age, v.salary }, + 2 + ); // Calculation of the mean value. This calculation will be performed in map-reduce manner. double[] mean = dataset.mean(); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/LocalDLCDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalBasedDatasetExample.java similarity index 87% rename from examples/src/main/java/org/apache/ignite/examples/ml/dlc/LocalDLCDatasetExample.java rename to examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalBasedDatasetExample.java index 055cf73ac057f..2d4a436a99a1a 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/LocalDLCDatasetExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalBasedDatasetExample.java @@ -15,21 +15,20 @@ * limitations under the License. */ -package org.apache.ignite.examples.ml.dlc; +package org.apache.ignite.examples.ml.dataset; import java.util.Arrays; import java.util.HashMap; import java.util.Map; import org.apache.ignite.Ignite; import org.apache.ignite.Ignition; -import org.apache.ignite.ml.dlc.dataset.DLCDataset; -import org.apache.ignite.ml.dlc.dataset.transformer.DLCTransformers; -import org.apache.ignite.ml.dlc.impl.local.MapBasedDLCFactory; +import org.apache.ignite.ml.dataset.DatasetFactory; +import org.apache.ignite.ml.dataset.api.SimpleDataset; /** * How to create a DLC dataset from an existing local data? */ -public class LocalDLCDatasetExample { +public class LocalBasedDatasetExample { /** Run example. */ public static void main(String[] args) { try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { @@ -44,12 +43,13 @@ public static void main(String[] args) { // context cache with specified transformation (it will be performed locally because partitions are on the // same nodes). In this case for every partition in upstream cache will be created labeled dataset partition // and this new partition will be filled with help of specified feature and label extractors. - DLCDataset dataset = new MapBasedDLCFactory<>(persons, 2) - .createDLC( - (data, size) -> null, - DLCTransformers.upstreamToDataset((k, v) -> new double[]{ v.age, v.salary }, 2), - DLCDataset::new - ); + + SimpleDataset dataset = DatasetFactory.createSimpleDataset( + persons, + 2, + (k, v) -> new double[]{ v.age, v.salary }, + 2 + ); // Calculation of the mean value. This calculation will be performed in map-reduce manner. double[] mean = dataset.mean(); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/package-info.java b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/package-info.java similarity index 95% rename from examples/src/main/java/org/apache/ignite/examples/ml/dlc/package-info.java rename to examples/src/main/java/org/apache/ignite/examples/ml/dataset/package-info.java index bf2c1998efece..325dc1d7991b6 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/package-info.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/package-info.java @@ -19,4 +19,4 @@ * * Machine learning Distributed Learning Context framework examples. */ -package org.apache.ignite.examples.ml.dlc; \ No newline at end of file +package org.apache.ignite.examples.ml.dataset; \ No newline at end of file diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificDataset.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificDataset.java deleted file mode 100644 index 43f835aea4de2..0000000000000 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificDataset.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * 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.examples.ml.dlc.transformer; - -import com.github.fommil.netlib.BLAS; -import java.util.Arrays; -import org.apache.ignite.ml.dlc.DLC; -import org.apache.ignite.ml.dlc.dataset.DLCWrapper; - -/** - * Algorithm-specific dataset which provides API allows to make gradient descent steps and maintain iteration number. - * - * @param type of an upstream value key - * @param type of an upstream value - */ -public class AlgorithmSpecificDataset extends DLCWrapper { - /** BlAS instance. */ - private static final BLAS blas = BLAS.getInstance(); - - /** - * Constructs a new instance of Distributed Learning Context wrapper - * - * @param delegate delegate which actually performs base functions - */ - public AlgorithmSpecificDataset( - DLC delegate) { - super(delegate); - } - - /** - * Make gradient step. - * - * @param x point to calculate gradient - * @return gradient - */ - public double[] makeGradientStep(double[] x) { - return compute((part, partIdx) -> { - AlgorithmSpecificRecoverableData recoverableData = part.getRecoverableData(); - AlgorithmSpecificReplicatedData replicatedData = part.getReplicatedData(); - - int iteration = replicatedData.getIteration(); - - double[] a = recoverableData.getA(); - double[] b = recoverableData.getB(); - int rows = recoverableData.getRows(); - int cols = recoverableData.getCols(); - double[] bCp = Arrays.copyOf(b, b.length); - - - blas.dgemv("N", rows, cols, 1.0, a, Math.max(0, rows), x, 1, -1.0, bCp, 1); - - double[] gradient = new double[cols]; - blas.dgemv("T", rows, cols, 1.0, a, Math.max(0, rows), bCp, 1, 0, gradient, 1); - - replicatedData.setIteration(iteration + 1); - - System.err.println("A = " + Arrays.toString(a) + ", B = " + Arrays.toString(b) + ", rows = " + rows + ", cols = " + cols + ", iter = " + iteration + ", g = " + Arrays.toString(gradient)); - - - return gradient; - }, this::reduce); - } - - /** - * Calculates sum of two vectors. - * - * @param a first vector - * @param b second vector - * @return sum of two vectors - */ - private double[] reduce(double[] a, double[] b) { - if (a == null) - return b; - - if (b == null) - return a; - - blas.daxpy(a.length, 1.0, a, 1, b, 1); - - return b; - } -} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificRecoverableData.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificRecoverableData.java deleted file mode 100644 index 5695d208dc073..0000000000000 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificRecoverableData.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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.examples.ml.dlc.transformer; - -/** - * Algorithm specific recoverable data consists of feature matrix supplemented by the column containing "1" and labels. - */ -public class AlgorithmSpecificRecoverableData implements AutoCloseable { - /** Matrix supplemented by the column containing "1". */ - private final double[] a; - - /** Number of rows. */ - private final int rows; - - /** Number of columns. */ - private final int cols; - - /** Labels. */ - private final double[] b; - - /** - * Constructs a new instance of algorithm specific recoverable data. - * - * @param a matrix supplemented by the column containing "1" - * @param b labels - */ - public AlgorithmSpecificRecoverableData(double[] a, int rows, int cols, double[] b) { - this.a = a; - this.rows = rows; - this.cols = cols; - this.b = b; - } - - /** */ - public double[] getA() { - return a; - } - - /** */ - public double[] getB() { - return b; - } - - /** */ - public int getRows() { - return rows; - } - - /** */ - public int getCols() { - return cols; - } - - /** */ - @Override public void close() { - // do nothing - } -} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificRecoverableDataTransformer.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificRecoverableDataTransformer.java deleted file mode 100644 index 3182741b7a7d9..0000000000000 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificRecoverableDataTransformer.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.examples.ml.dlc.transformer; - -import org.apache.ignite.ml.dlc.dataset.part.DLCLabeledDatasetPartitionRecoverable; -import org.apache.ignite.ml.math.functions.IgniteFunction; - -/** - * Transformer which transforms dataset recoverable data to algorithm specific recoverable data by supplementing column - * with value "1". - */ -public class AlgorithmSpecificRecoverableDataTransformer - implements IgniteFunction { - /** */ - private static final long serialVersionUID = 2292486552367922497L; - - /** {@inheritDoc} */ - @Override public AlgorithmSpecificRecoverableData apply(DLCLabeledDatasetPartitionRecoverable recoverable) { - int rows = recoverable.getRows(); - int cols = recoverable.getCols(); - - // Add first column with values equal to 1.0. - double[] supplementedMatrix = new double[rows * (cols + 1)]; - for (int i = 0; i < rows; i++) - supplementedMatrix[i] = 1.0; - System.arraycopy(recoverable.getFeatures(), 0, supplementedMatrix, rows, rows * cols); - - return new AlgorithmSpecificRecoverableData(supplementedMatrix, rows, cols + 1, recoverable.getLabels()); - } -} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/DLCTransformerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/DLCTransformerExample.java deleted file mode 100644 index 11c41b6be0ec5..0000000000000 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/DLCTransformerExample.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * 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.examples.ml.dlc.transformer; - -import com.github.fommil.netlib.BLAS; -import java.util.Arrays; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.Ignition; -import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.ml.dlc.dataset.transformer.DLCTransformers; -import org.apache.ignite.ml.dlc.impl.cache.CacheBasedDLCFactory; - -/** How to transform DLC into algorithm-specific view? */ -public class DLCTransformerExample { - /** Run example. */ - public static void main(String[] args) { - BLAS bas = BLAS.getInstance(); - - - - try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { - System.out.println(">>> D-Learn Cache Dataset example started."); - - IgniteCache persons = createCache(ignite); - - // Initialization of the d-learn context. After this step context cache will be created with partitions - // placed on the same nodes as the upstream Ignite Cache. - - // Loading of the d-learn context. During this step data will be transferred from the upstream cache to - // context cache with specified transformation (it will be performed locally because partitions are on the - // same nodes). In this case for every partition in upstream cache will be created labeled dataset partition - // and this new partition will be filled with help of specified feature and label extractors. - - AlgorithmSpecificDataset dataset = new CacheBasedDLCFactory<>(ignite, persons) - .createDLC( - new AlgorithmSpecificReplicatedDataTransformer<>(), - DLCTransformers.upstreamToLabeledDataset( - (k, v) -> new double[]{ v.age }, - (k, v) -> v.salary, - 1 - ).andThen(new AlgorithmSpecificRecoverableDataTransformer()), - AlgorithmSpecificDataset::new - ); - - double[] x = new double[2]; - - for (int iter = 0; iter < 100; iter++) { - double[] gradient = dataset.makeGradientStep(x); - System.out.println("Point : " + Arrays.toString(x) + ", gradient : " + Arrays.toString(gradient)); - for (int i = 0; i < x.length; i++) - x[i] -= 0.01 * gradient[i]; - } - - System.out.println("New point : " + Arrays.toString(x)); - - System.out.println(">>> D-Learn Cache Dataset example completed."); - } - } - - /** */ - private static IgniteCache createCache(Ignite ignite) { - CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); - cacheConfiguration.setName("PERSONS"); - cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 1)); - IgniteCache persons = ignite.createCache(cacheConfiguration); - persons.put(1, new Person("Mike", 10, 100)); - persons.put(2, new Person("John", 20, 200)); - persons.put(3, new Person("George", 30, 300)); - persons.put(4, new Person("Karl", 40, 400)); - return persons; - } - - /** */ - private static class Person { - /** */ - private final String name; - - /** */ - private final double age; - - /** */ - private final double salary; - - /** */ - public Person(String name, double age, double salary) { - this.name = name; - this.age = age; - this.salary = salary; - } - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/Dataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/Dataset.java new file mode 100644 index 0000000000000..a3c40947abb28 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/Dataset.java @@ -0,0 +1,210 @@ +/* + * 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.ml.dataset; + +import java.io.Serializable; +import org.apache.ignite.ml.math.functions.IgniteBiConsumer; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; +import org.apache.ignite.ml.math.functions.IgniteConsumer; +import org.apache.ignite.ml.math.functions.IgniteFunction; +import org.apache.ignite.ml.math.functions.IgniteTriConsumer; +import org.apache.ignite.ml.math.functions.IgniteTriFunction; + +/** + * Distributed Learning Context provides the API which allows to perform iterative computation tasks on a distributed + * datasets. Every computation performed via Distributed Learning Context works with {@link DLCPartition} which consists + * of replicated data and recoverable data. Computation task can modify these segments to maintain the iterative + * algorithm context. + * + * @param + * @param + */ +public interface Dataset extends AutoCloseable { + /** + * Computes the given function on every partition in the current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param map + * @param reduce + * @param identity + * @param + * @return + */ + public R computeWithCtx(IgniteTriFunction map, IgniteBinaryOperator reduce, R identity); + + /** + * Computes the given function on every partition in the current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param map + * @param reduce + * @param identity + * @param + * @return + */ + public R compute(IgniteBiFunction map, IgniteBinaryOperator reduce, R identity); + + /** + * Computes the given function on every partition in the current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param map + * @param reduce + * @param + * @return + */ + default public R computeWithCtx(IgniteTriFunction map, IgniteBinaryOperator reduce) { + return computeWithCtx(map, reduce, null); + } + + /** + * Computes the given function on every partition in the current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param map + * @param reduce + * @param + * @return + */ + default public R compute(IgniteBiFunction map, IgniteBinaryOperator reduce) { + return compute(map, reduce, null); + } + + /** + * Computes the given function on every partition in the current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param map + * @param reduce + * @param identity + * @param + * @return + */ + default public R computeWithCtx(IgniteBiFunction map, IgniteBinaryOperator reduce, R identity) { + return computeWithCtx((ctx, data, partIdx) -> map.apply(ctx, data), reduce, identity); + } + + /** + * Computes the given function on every partition in the current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param map + * @param reduce + * @param identity + * @param + * @return + */ + default public R compute(IgniteFunction map, IgniteBinaryOperator reduce, R identity) { + return compute((data, partIdx) -> map.apply(data), reduce, identity); + } + + /** + * Computes the given function on every partition in the current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param map + * @param reduce + * @param + * @return + */ + default public R computeWithCtx(IgniteBiFunction map, IgniteBinaryOperator reduce) { + return computeWithCtx((ctx, data, partIdx) -> map.apply(ctx, data), reduce); + } + + /** + * Computes the given function on every partition in the current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param map + * @param reduce + * @param + * @return + */ + default public R compute(IgniteFunction map, IgniteBinaryOperator reduce) { + return compute((data, partIdx) -> map.apply(data), reduce); + } + + /** + * Computes the given function on every partition in the current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param map + */ + default public void computeWithCtx(IgniteTriConsumer map) { + computeWithCtx((ctx, data, partIdx) -> { + map.accept(ctx, data, partIdx); + return null; + }, (a, b) -> null); + } + + /** + * Computes the given function on every partition in the current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param map + */ + default public void compute(IgniteBiConsumer map) { + compute((data, partIdx) -> { + map.accept(data, partIdx); + return null; + }, (a, b) -> null); + } + + /** + * Computes the given function on every partition in the current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param map + */ + default public void computeWithCtx(IgniteBiConsumer map) { + computeWithCtx((ctx, data, partIdx) -> map.accept(ctx, data)); + } + + /** + * Computes the given function on every partition in the current learning context independently and then reduces + * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes + * where partitions are placed and do not involve network subsystem where it's possible. + * + * @param map + */ + default public void compute(IgniteConsumer map) { + compute((data, partIdx) -> map.accept(data)); + } + + /** + * + * @param wrapper + * @param + * @return + */ + default public > I wrap(IgniteFunction, I> wrapper) { + return wrapper.apply(this); + } +} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificReplicatedData.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java similarity index 62% rename from examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificReplicatedData.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java index 9400a298febbe..102e372217650 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificReplicatedData.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java @@ -15,27 +15,19 @@ * limitations under the License. */ -package org.apache.ignite.examples.ml.dlc.transformer; +package org.apache.ignite.ml.dataset; import java.io.Serializable; /** - * Algorithm-specific replicated data maintaining number of iteration. + * + * @param + * @param */ -public class AlgorithmSpecificReplicatedData implements Serializable { - /** */ - private static final long serialVersionUID = 7593887973722933436L; - - /** Number of iteration. */ - private int iteration; - - /** */ - public int getIteration() { - return iteration; - } - - /** */ - public void setIteration(int iteration) { - this.iteration = iteration; - } +public interface DatasetBuilder { + /** + * + * @return + */ + public Dataset build(); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetFactory.java new file mode 100644 index 0000000000000..2bf5336f65c0b --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetFactory.java @@ -0,0 +1,238 @@ +/* + * 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.ml.dataset; + +import java.io.Serializable; +import java.util.Map; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.ml.dataset.api.SimpleDataset; +import org.apache.ignite.ml.dataset.api.SimpleLabeledDataset; +import org.apache.ignite.ml.dataset.api.builder.context.EmptyContextBuilder; +import org.apache.ignite.ml.dataset.api.builder.data.SimpleDatasetDataBuilder; +import org.apache.ignite.ml.dataset.api.builder.data.SimpleLabeledDatasetDataBuilder; +import org.apache.ignite.ml.dataset.api.context.EmptyContext; +import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +public class DatasetFactory { + /** + * + * + * @param ignite + * @param upstreamCache + * @param partCtxBuilder + * @param partDataBuilder + * @param + * @param + * @param + * @param + * @return + */ + public static Dataset create( + Ignite ignite, IgniteCache upstreamCache, PartitionContextBuilder partCtxBuilder, + PartitionDataBuilder partDataBuilder) { + return new CacheBasedDatasetBuilder<>(ignite, upstreamCache, partCtxBuilder, partDataBuilder).build(); + } + + /** + * + * + * @param ignite + * @param upstreamCache + * @param partCtxBuilder + * @param featureExtractor + * @param cols + * @param + * @param + * @param + * @return + */ + public static SimpleDataset createSimpleDataset(Ignite ignite, + IgniteCache upstreamCache, PartitionContextBuilder partCtxBuilder, + IgniteBiFunction featureExtractor, int cols) { + return create( + ignite, + upstreamCache, + partCtxBuilder, + new SimpleDatasetDataBuilder<>(featureExtractor, cols) + ).wrap(SimpleDataset::new); + } + + /** + * + * + * @param ignite + * @param upstreamCache + * @param partCtxBuilder + * @param featureExtractor + * @param lbExtractor + * @param cols + * @param + * @param + * @param + * @return + */ + public static SimpleLabeledDataset createSimpleLabeledDataset(Ignite ignite, + IgniteCache upstreamCache, PartitionContextBuilder partCtxBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor, int cols) { + return create( + ignite, + upstreamCache, + partCtxBuilder, + new SimpleLabeledDatasetDataBuilder<>(featureExtractor, lbExtractor, cols) + ).wrap(SimpleLabeledDataset::new); + } + + /** + * + * + * @param ignite + * @param upstreamCache + * @param featureExtractor + * @param cols + * @param + * @param + * @return + */ + public static SimpleDataset createSimpleDataset(Ignite ignite, IgniteCache upstreamCache, + IgniteBiFunction featureExtractor, int cols) { + return createSimpleDataset(ignite, upstreamCache, new EmptyContextBuilder<>(), featureExtractor, cols); + } + + /** + * + * + * @param ignite + * @param upstreamCache + * @param featureExtractor + * @param lbExtractor + * @param cols + * @param + * @param + * @return + */ + public static SimpleLabeledDataset createSimpleLabeledDataset(Ignite ignite, + IgniteCache upstreamCache, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor, int cols) { + return createSimpleLabeledDataset(ignite, upstreamCache, new EmptyContextBuilder<>(), featureExtractor, lbExtractor, cols); + } + + /** + * + * + * @param upstreamMap + * @param partitions + * @param partCtxBuilder + * @param partDataBuilder + * @param + * @param + * @param + * @param + * @return + */ + public static Dataset create( + Map upstreamMap, int partitions, PartitionContextBuilder partCtxBuilder, + PartitionDataBuilder partDataBuilder) { + return new LocalDatasetBuilder<>(upstreamMap, partitions, partCtxBuilder, partDataBuilder).build(); + } + + /** + * + * + * @param upstreamMap + * @param partitions + * @param partCtxBuilder + * @param featureExtractor + * @param cols + * @param + * @param + * @param + * @return + */ + public static SimpleDataset createSimpleDataset(Map upstreamMap, + int partitions, PartitionContextBuilder partCtxBuilder, + IgniteBiFunction featureExtractor, int cols) { + return create( + upstreamMap, + partitions, + partCtxBuilder, + new SimpleDatasetDataBuilder<>(featureExtractor, cols) + ).wrap(SimpleDataset::new); + } + + /** + * + * + * @param upstreamMap + * @param partitions + * @param partCtxBuilder + * @param featureExtractor + * @param lbExtractor + * @param cols + * @param + * @param + * @param + * @return + */ + public static SimpleLabeledDataset createSimpleLabeledDataset( + Map upstreamMap, int partitions, PartitionContextBuilder partCtxBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor, int cols) { + return create( + upstreamMap, + partitions, + partCtxBuilder, + new SimpleLabeledDatasetDataBuilder<>(featureExtractor, lbExtractor, cols) + ).wrap(SimpleLabeledDataset::new); + } + + /** + * + * + * @param upstreamMap + * @param partitions + * @param featureExtractor + * @param cols + * @param + * @param + * @return + */ + public static SimpleDataset createSimpleDataset(Map upstreamMap, int partitions, + IgniteBiFunction featureExtractor, int cols) { + return createSimpleDataset(upstreamMap, partitions, new EmptyContextBuilder<>(), featureExtractor, cols); + } + + /** + * + * + * @param upstreamMap + * @param partitions + * @param featureExtractor + * @param lbExtractor + * @param cols + * @param + * @param + * @return + */ + public static SimpleLabeledDataset createSimpleLabeledDataset(Map upstreamMap, + int partitions, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor, + int cols) { + return createSimpleLabeledDataset(upstreamMap, partitions, new EmptyContextBuilder<>(), featureExtractor, lbExtractor, cols); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCLabeledDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionContextBuilder.java similarity index 55% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCLabeledDataset.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionContextBuilder.java index ed282c06d5d5a..10e735ce988bf 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCLabeledDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionContextBuilder.java @@ -15,27 +15,29 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlc.dataset; +package org.apache.ignite.ml.dataset; import java.io.Serializable; -import org.apache.ignite.ml.dlc.DLC; -import org.apache.ignite.ml.dlc.dataset.part.DLCLabeledDatasetPartitionRecoverable; +import java.util.Iterator; +import org.apache.ignite.ml.math.functions.IgniteFunction; /** - * Dataset provides API to work with labeled dataset. * - * @param type of an upstream value key - * @param type of an upstream value + * @param + * @param + * @param */ -public class DLCLabeledDataset - extends DLCWrapper { +@FunctionalInterface +public interface PartitionContextBuilder { /** - * Constructs a new instance of Distributed Learning Context wrapper * - * @param delegate delegate which actually performs base functions + * @param upstreamData + * @param upstreamDataSize + * @return */ - public DLCLabeledDataset( - DLC delegate) { - super(delegate); + public C build(Iterator> upstreamData, long upstreamDataSize); + + default public PartitionContextBuilder andThen(IgniteFunction fun) { + return (upstreamData, upstreamDataSize) -> fun.apply(build(upstreamData, upstreamDataSize)); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java new file mode 100644 index 0000000000000..6f65f9ce8ef54 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java @@ -0,0 +1,51 @@ +/* + * 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.ml.dataset; + +import java.io.Serializable; +import java.util.Iterator; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +/** + * + * @param + * @param + * @param + * @param + */ +@FunctionalInterface +public interface PartitionDataBuilder extends Serializable { + /** + * + * @param upstreamData + * @param upstreamDataSize + * @param ctx + * @return + */ + public D build(Iterator> upstreamData, long upstreamDataSize, C ctx); + + /** + * + * @param fun + * @param + * @return + */ + default public PartitionDataBuilder andThen(IgniteBiFunction fun) { + return (upstreamData, upstreamDataSize, ctx) -> fun.apply(build(upstreamData, upstreamDataSize, ctx), ctx); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCUpstreamEntry.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionUpstreamEntry.java similarity index 91% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCUpstreamEntry.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionUpstreamEntry.java index 0e8ff5cbbed26..4c7288da30682 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCUpstreamEntry.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionUpstreamEntry.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlc; +package org.apache.ignite.ml.dataset; /** * Distributed Learning Context upstream entry. @@ -23,7 +23,7 @@ * @param type of an upstream value key * @param type of an upstream value */ -public class DLCUpstreamEntry { +public class PartitionUpstreamEntry { /** Key. */ private final K key; @@ -36,7 +36,7 @@ public class DLCUpstreamEntry { * @param key key * @param val value */ - public DLCUpstreamEntry(K key, V val) { + public PartitionUpstreamEntry(K key, V val) { this.key = key; this.val = val; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/DatasetWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/DatasetWrapper.java new file mode 100644 index 0000000000000..00e7a853e4f27 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/DatasetWrapper.java @@ -0,0 +1,46 @@ +/* + * 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.ml.dataset.api; + +import java.io.Serializable; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; +import org.apache.ignite.ml.math.functions.IgniteTriFunction; + +public class DatasetWrapper implements Dataset { + + protected final Dataset delegate; + + public DatasetWrapper(Dataset delegate) { + this.delegate = delegate; + } + + @Override public R computeWithCtx(IgniteTriFunction map, IgniteBinaryOperator reduce, + R identity) { + return delegate.computeWithCtx(map, reduce, identity); + } + + @Override public R compute(IgniteBiFunction map, IgniteBinaryOperator reduce, R identity) { + return delegate.compute(map, reduce, identity); + } + + @Override public void close() throws Exception { + delegate.close(); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java similarity index 75% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCDataset.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java index 6347db386964f..675ea8b4d0d22 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java @@ -15,28 +15,22 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlc.dataset; +package org.apache.ignite.ml.dataset.api; import com.github.fommil.netlib.BLAS; import java.io.Serializable; -import org.apache.ignite.ml.dlc.DLC; -import org.apache.ignite.ml.dlc.dataset.part.DLCDatasetPartitionRecoverable; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.api.data.SimpleDatasetData; -/** - * Dataset provides API to work with dataset. - * - * @param type of an upstream value key - * @param type of an upstream value - */ -public class DLCDataset extends DLCWrapper { +public class SimpleDataset extends DatasetWrapper { /** BLAS (Basic Linear Algebra Subprograms) instance. */ private static final BLAS blas = BLAS.getInstance(); + /** - * Constructs a new instance of Distributed Learning Context wrapper * - * @param delegate delegate which actually performs base functions + * @param delegate */ - public DLCDataset(DLC delegate) { + public SimpleDataset(Dataset delegate) { super(delegate); } @@ -46,12 +40,10 @@ public DLCDataset(DLC delegate) { * @return mean values */ public double[] mean() { - ValueWithCount res = delegate.compute((part, partIdx) -> { - DLCDatasetPartitionRecoverable recoverableData = part.getRecoverableData(); - - double[] features = recoverableData.getFeatures(); - int rows = recoverableData.getRows(); - int cols = recoverableData.getCols(); + ValueWithCount res = delegate.compute((data, partIdx) -> { + double[] features = data.getFeatures(); + int rows = data.getRows(); + int cols = data.getCols(); double[] y = new double[cols]; @@ -77,12 +69,10 @@ public double[] mean() { */ public double[] std() { double[] mean = mean(); - ValueWithCount res = delegate.compute(part -> { - DLCDatasetPartitionRecoverable recoverableData = part.getRecoverableData(); - - double[] features = recoverableData.getFeatures(); - int rows = recoverableData.getRows(); - int cols = recoverableData.getCols(); + ValueWithCount res = delegate.compute(data -> { + double[] features = data.getFeatures(); + int rows = data.getRows(); + int cols = data.getCols(); double[] y = new double[cols]; @@ -110,12 +100,10 @@ public double[] std() { */ public double[][] cov() { double[] mean = mean(); - ValueWithCount res = delegate.compute(part -> { - DLCDatasetPartitionRecoverable recoverableData = part.getRecoverableData(); - - double[] features = recoverableData.getFeatures(); - int rows = recoverableData.getRows(); - int cols = recoverableData.getCols(); + ValueWithCount res = delegate.compute(data -> { + double[] features = data.getFeatures(); + int rows = data.getRows(); + int cols = data.getCols(); double[][] y = new double[cols][cols]; diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificReplicatedDataTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleLabeledDataset.java similarity index 50% rename from examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificReplicatedDataTransformer.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleLabeledDataset.java index 8ca2af818b109..09bb0b5eb2e15 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlc/transformer/AlgorithmSpecificReplicatedDataTransformer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleLabeledDataset.java @@ -15,24 +15,15 @@ * limitations under the License. */ -package org.apache.ignite.examples.ml.dlc.transformer; +package org.apache.ignite.ml.dataset.api; -import org.apache.ignite.ml.dlc.DLCPartitionReplicatedTransformer; -import org.apache.ignite.ml.dlc.DLCUpstreamEntry; +import java.io.Serializable; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.api.data.SimpleLabeledDatasetData; -/** - * Transformer which transforms upstream data into algorithm-specific replicated data. - * - * @param type of an upstream value key - * @param type of an upstream value - */ -public class AlgorithmSpecificReplicatedDataTransformer implements DLCPartitionReplicatedTransformer { - /** */ - private static final long serialVersionUID = -3720585607267757357L; +public class SimpleLabeledDataset extends DatasetWrapper { - /** {@inheritDoc} */ - @Override public AlgorithmSpecificReplicatedData transform(Iterable> upstreamData, - Long upstreamDataSize) { - return new AlgorithmSpecificReplicatedData(); + public SimpleLabeledDataset(Dataset delegate) { + super(delegate); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/EmptyContextBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/EmptyContextBuilder.java new file mode 100644 index 0000000000000..ddef16c3589de --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/EmptyContextBuilder.java @@ -0,0 +1,30 @@ +/* + * 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.ml.dataset.api.builder.context; + +import java.util.Iterator; +import org.apache.ignite.ml.dataset.PartitionContextBuilder; +import org.apache.ignite.ml.dataset.PartitionUpstreamEntry; +import org.apache.ignite.ml.dataset.api.context.EmptyContext; + +public class EmptyContextBuilder implements PartitionContextBuilder { + + @Override public EmptyContext build(Iterator> upstreamData, long upstreamDataSize) { + return new EmptyContext(); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleDatasetDataBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleDatasetDataBuilder.java new file mode 100644 index 0000000000000..503a022a97873 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleDatasetDataBuilder.java @@ -0,0 +1,59 @@ +/* + * 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.ml.dataset.api.builder.data; + +import java.io.Serializable; +import java.util.Iterator; +import org.apache.ignite.ml.dataset.PartitionDataBuilder; +import org.apache.ignite.ml.dataset.PartitionUpstreamEntry; +import org.apache.ignite.ml.dataset.api.data.SimpleDatasetData; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +public class SimpleDatasetDataBuilder + implements PartitionDataBuilder { + + private final IgniteBiFunction featureExtractor; + + private final int cols; + + public SimpleDatasetDataBuilder(IgniteBiFunction featureExtractor, int cols) { + this.featureExtractor = featureExtractor; + this.cols = cols; + } + + /** */ + @Override public SimpleDatasetData build(Iterator> upstreamData, long upstreamDataSize, + C ctx) { + double[] features = new double[Math.toIntExact(upstreamDataSize * cols)]; + + int ptr = 0; + while (upstreamData.hasNext()) { + PartitionUpstreamEntry entry = upstreamData.next(); + double[] row = featureExtractor.apply(entry.getKey(), entry.getValue()); + + assert row.length == cols; + + for (int i = 0; i < cols; i++) + features[Math.toIntExact(i * upstreamDataSize) + ptr] = row[i]; + + ptr++; + } + + return new SimpleDatasetData(features, Math.toIntExact(upstreamDataSize), cols); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleLabeledDatasetDataBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleLabeledDatasetDataBuilder.java new file mode 100644 index 0000000000000..6becec77ab3d2 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleLabeledDatasetDataBuilder.java @@ -0,0 +1,66 @@ +/* + * 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.ml.dataset.api.builder.data; + +import java.io.Serializable; +import java.util.Iterator; +import org.apache.ignite.ml.dataset.PartitionDataBuilder; +import org.apache.ignite.ml.dataset.PartitionUpstreamEntry; +import org.apache.ignite.ml.dataset.api.data.SimpleLabeledDatasetData; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +public class SimpleLabeledDatasetDataBuilder + implements PartitionDataBuilder { + + private final IgniteBiFunction featureExtractor; + + private final IgniteBiFunction lbExtractor; + + private final int cols; + + public SimpleLabeledDatasetDataBuilder(IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor, int cols) { + this.featureExtractor = featureExtractor; + this.lbExtractor = lbExtractor; + this.cols = cols; + } + + /** */ + @Override public SimpleLabeledDatasetData build(Iterator> upstreamData, + long upstreamDataSize, C ctx) { + double[] features = new double[Math.toIntExact(upstreamDataSize * cols)]; + double[] labels = new double[Math.toIntExact(upstreamDataSize)]; + + int ptr = 0; + while (upstreamData.hasNext()) { + PartitionUpstreamEntry entry = upstreamData.next(); + double[] row = featureExtractor.apply(entry.getKey(), entry.getValue()); + + assert row.length == cols; + + for (int i = 0; i < cols; i++) + features[Math.toIntExact(i * upstreamDataSize) + ptr] = row[i]; + + labels[ptr] = lbExtractor.apply(entry.getKey(), entry.getValue()); + + ptr++; + } + + return new SimpleLabeledDatasetData(features, Math.toIntExact(upstreamDataSize), cols, labels); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/EmptyContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/EmptyContext.java new file mode 100644 index 0000000000000..5cb436dcb2ebe --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/EmptyContext.java @@ -0,0 +1,23 @@ +/* + * 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.ml.dataset.api.context; + +import java.io.Serializable; + +public class EmptyContext implements Serializable { +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/DLCDatasetPartitionRecoverable.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleDatasetData.java similarity index 72% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/DLCDatasetPartitionRecoverable.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleDatasetData.java index 15969b00dac69..7d5fb80433487 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/DLCDatasetPartitionRecoverable.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleDatasetData.java @@ -15,13 +15,10 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlc.dataset.part; +package org.apache.ignite.ml.dataset.api.data; -/** - * Recoverable part of a DLC dataset partition contains matrix with features in a dense flat column-major format. - */ -public class DLCDatasetPartitionRecoverable implements AutoCloseable { - /** Matrix with features in a dense flat column-major format. */ +public class SimpleDatasetData implements AutoCloseable { + /** Matrix of features in a dense flat column-major format. */ private final double[] features; /** Number of rows. */ @@ -31,13 +28,13 @@ public class DLCDatasetPartitionRecoverable implements AutoCloseable { private final int cols; /** - * Constructs a new instance of recoverable data of DLC dataset partition. + * Constructs a new instance of recoverable data of DLC labeled dataset partition. * - * @param features matrix with features in a dense flat column-major format + * @param features matrix of features in a dense flat column-major format * @param rows number of rows * @param cols number of columns */ - public DLCDatasetPartitionRecoverable(double[] features, int rows, int cols) { + public SimpleDatasetData(double[] features, int rows, int cols) { this.features = features; this.rows = rows; this.cols = cols; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/DLCLabeledDatasetPartitionRecoverable.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleLabeledDatasetData.java similarity index 63% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/DLCLabeledDatasetPartitionRecoverable.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleLabeledDatasetData.java index 1eaade1856504..2dbbfa1fa1962 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/part/DLCLabeledDatasetPartitionRecoverable.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleLabeledDatasetData.java @@ -15,14 +15,10 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlc.dataset.part; +package org.apache.ignite.ml.dataset.api.data; -/** - * Recoverable part of a DLC labeled dataset partition contains matrix of features in a dense flat column-major format - * and vector of values. - */ -public class DLCLabeledDatasetPartitionRecoverable implements AutoCloseable { - /** Matrix of features in a dense flat column-major format. */ +public class SimpleLabeledDatasetData implements AutoCloseable { + /** Matrix with features in a dense flat column-major format. */ private final double[] features; /** Number of rows. */ @@ -32,17 +28,16 @@ public class DLCLabeledDatasetPartitionRecoverable implements AutoCloseable { private final int cols; /** Vector with labels. */ - private double[] labels; + private final double[] labels; /** - * Constructs a new instance of recoverable data of DLC labeled dataset partition. * - * @param features matrix of features in a dense flat column-major format - * @param rows number of rows - * @param cols number of columns - * @param labels vector with labels + * @param features + * @param rows + * @param cols + * @param labels */ - public DLCLabeledDatasetPartitionRecoverable(double[] features, int rows, int cols, double[] labels) { + public SimpleLabeledDatasetData(double[] features, int rows, int cols, double[] labels) { this.features = features; this.rows = rows; this.cols = cols; @@ -64,16 +59,10 @@ public int getCols() { return cols; } - /** */ public double[] getLabels() { return labels; } - /** */ - public void setLabels(double[] labels) { - this.labels = labels; - } - /** {@inheritDoc} */ @Override public void close() { // do nothing diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java new file mode 100644 index 0000000000000..de4eb5f3e3565 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java @@ -0,0 +1,146 @@ +/* + * 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.ml.dataset.impl.cache; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collection; +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.PartitionDataBuilder; +import org.apache.ignite.ml.dataset.impl.cache.util.ComputeUtils; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; +import org.apache.ignite.ml.math.functions.IgniteFunction; +import org.apache.ignite.ml.math.functions.IgniteTriFunction; + +/** + * + * @param + * @param + * @param + * @param + */ +public class CacheBasedDataset + implements Dataset { + /** */ + private static final int RETRIES = 100; + + /** */ + private static final int RETRY_INTERVAL = 500; + + /** */ + private final Ignite ignite; + + /** */ + private final IgniteCache upstreamCache; + + /** */ + private final IgniteCache datasetCache; + + /** */ + private final PartitionDataBuilder partDataBuilder; + + /** */ + private final UUID datasetId; + + /** + * + * @param ignite + * @param upstreamCache + * @param datasetCache + * @param partDataBuilder + * @param datasetId + */ + public CacheBasedDataset(Ignite ignite, IgniteCache upstreamCache, + IgniteCache datasetCache, PartitionDataBuilder partDataBuilder, + UUID datasetId) { + this.ignite = ignite; + this.upstreamCache = upstreamCache; + this.datasetCache = datasetCache; + this.partDataBuilder = partDataBuilder; + this.datasetId = datasetId; + } + + /** {@inheritDoc} */ + @Override public R computeWithCtx(IgniteTriFunction map, IgniteBinaryOperator reduce, R identity) { + String upstreamCacheName = upstreamCache.getName(); + String datasetCacheName = datasetCache.getName(); + + return computeForAllPartitions(part -> { + C ctx = ComputeUtils.getContext(Ignition.localIgnite(), datasetCacheName, part); + + D data = ComputeUtils.getData( + Ignition.localIgnite(), + upstreamCacheName, + datasetCacheName, + datasetId, + part, + partDataBuilder + ); + + return map.apply(ctx, data, part); + }, reduce, identity); + } + + /** {@inheritDoc} */ + @Override public R compute(IgniteBiFunction map, IgniteBinaryOperator reduce, R identity) { + String upstreamCacheName = upstreamCache.getName(); + String datasetCacheName = datasetCache.getName(); + + return computeForAllPartitions(part -> { + D data = ComputeUtils.getData( + Ignition.localIgnite(), + upstreamCacheName, + datasetCacheName, + datasetId, + part, + partDataBuilder + ); + + return map.apply(data, part); + }, reduce, identity); + } + + /** {@inheritDoc} */ + @Override public void close() { + datasetCache.destroy(); + } + + /** + * + * @param fun + * @param reduce + * @param identity + * @param + * @return + */ + private R computeForAllPartitions(IgniteFunction fun, IgniteBinaryOperator reduce, R identity) { + Collection cacheNames = Arrays.asList(datasetCache.getName(), upstreamCache.getName()); + Collection results = ComputeUtils.affinityCallWithRetries(ignite, cacheNames, fun, RETRIES, RETRY_INTERVAL); + + R res = identity; + for (R partRes : results) + res = reduce.apply(res, partRes); + + return res; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java new file mode 100644 index 0000000000000..69b8f60baf565 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java @@ -0,0 +1,90 @@ +/* + * 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.ml.dataset.impl.cache; + +import java.io.Serializable; +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.affinity.AffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.dataset.PartitionContextBuilder; +import org.apache.ignite.ml.dataset.PartitionDataBuilder; +import org.apache.ignite.ml.dataset.impl.cache.util.ComputeUtils; +import org.apache.ignite.ml.dataset.impl.cache.util.DatasetAffinityFunctionWrapper; + +/** + * + * @param + * @param + * @param + * @param + */ +public class CacheBasedDatasetBuilder + implements DatasetBuilder { + + private static final int RETRIES = 100; + + private static final int RETRY_INTERVAL = 500; + + private static final String DATASET_CACHE_TEMPLATE = "%s_DATASET_%s"; + + /** Ignite instance. */ + private final Ignite ignite; + + private final IgniteCache upstreamCache; + + private final PartitionContextBuilder partCtxBuilder; + + private final PartitionDataBuilder partDataBuilder; + + public CacheBasedDatasetBuilder(Ignite ignite, IgniteCache upstreamCache, + PartitionContextBuilder partCtxBuilder, PartitionDataBuilder partDataBuilder) { + this.ignite = ignite; + this.upstreamCache = upstreamCache; + this.partCtxBuilder = partCtxBuilder; + this.partDataBuilder = partDataBuilder; + } + + @SuppressWarnings("unchecked") + @Override public Dataset build() { + UUID datasetId = UUID.randomUUID(); + + CacheConfiguration upstreamCacheConfiguration = upstreamCache.getConfiguration(CacheConfiguration.class); + AffinityFunction upstreamCacheAffinity = upstreamCacheConfiguration.getAffinity(); + + CacheConfiguration datasetCacheConfiguration = new CacheConfiguration<>(); + datasetCacheConfiguration.setName(String.format(DATASET_CACHE_TEMPLATE, upstreamCache.getName(), datasetId)); + datasetCacheConfiguration.setAffinity(new DatasetAffinityFunctionWrapper(upstreamCacheAffinity)); + + IgniteCache datasetCache = ignite.createCache(datasetCacheConfiguration); + + ComputeUtils.initContext( + ignite, + upstreamCache.getName(), + datasetCache.getName(), + partCtxBuilder, + RETRIES, + RETRY_INTERVAL + ); + + return new CacheBasedDataset<>(ignite, upstreamCache, datasetCache, partDataBuilder, datasetId); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java new file mode 100644 index 0000000000000..17bcaf217a5af --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java @@ -0,0 +1,243 @@ +/* + * 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.ml.dataset.impl.cache.util; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.BitSet; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.locks.LockSupport; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.cluster.ClusterGroup; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.ml.dataset.PartitionContextBuilder; +import org.apache.ignite.ml.dataset.PartitionDataBuilder; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +/** + * + */ +public class ComputeUtils { + + private static final String DATA_STORAGE_KEY = "part_data_storage_%s"; + + /** + * @param ignite + * @param cacheNames + * @param fun + * @param retries + * @param interval + * @param + * @return + */ + public static Collection affinityCallWithRetries(Ignite ignite, Collection cacheNames, + IgniteFunction fun, int retries, int interval) { + assert cacheNames.size() > 0; + assert interval >= 0; + + String primaryCache = cacheNames.iterator().next(); + + Affinity affinity = ignite.affinity(primaryCache); + int partitions = affinity.partitions(); + + BitSet completionFlags = new BitSet(partitions); + Collection results = new ArrayList<>(); + + for (int t = 0; t < retries; t++) { + ClusterGroup clusterGrp = ignite.cluster().forDataNodes(primaryCache); + + // Sends jobs. + Map> futures = new HashMap<>(); + for (int part = 0; part < partitions; part++) + if (!completionFlags.get(part)) { + final int currPart = part; + + futures.put(currPart, ignite.compute(clusterGrp).affinityCallAsync(cacheNames, currPart, () -> { + checkAllPartitionsAvailable(Ignition.localIgnite(), cacheNames, currPart); + + return fun.apply(currPart); + })); + } + + // Collects results. + for (int part : futures.keySet()) { + try { + R res = futures.get(part).get(); + results.add(res); + completionFlags.set(part); + } + catch (PartitionNotFoundException ignore) { + } + } + + if (completionFlags.cardinality() == partitions) + return results; + + LockSupport.parkNanos(interval * 1_000_000); + } + + throw new IllegalStateException(); + } + + /** + * @param ignite + * @param cacheNames + * @param fun + * @param retries + * @param + * @return + */ + public static Collection affinityCallWithRetries(Ignite ignite, Collection cacheNames, + IgniteFunction fun, int retries) { + return affinityCallWithRetries(ignite, cacheNames, fun, retries, 0); + } + + /** + * @param ignite + * @param upstreamCacheName + * @param datasetCacheName + * @param learningCtxId + * @param part + * @param partDataBuilder + * @param + * @param + * @param + * @param + * @return + */ + @SuppressWarnings("unchecked") + public static D getData(Ignite ignite, + String upstreamCacheName, String datasetCacheName, UUID learningCtxId, int part, + PartitionDataBuilder partDataBuilder) { + + PartitionDataStorage dataStorage = (PartitionDataStorage)ignite + .cluster() + .nodeLocalMap() + .computeIfAbsent(String.format(DATA_STORAGE_KEY, learningCtxId), key -> new PartitionDataStorage()); + + Object data = dataStorage.computeDataIfAbsent(part, () -> { + IgniteCache learningCtxCache = ignite.cache(datasetCacheName); + C ctx = learningCtxCache.get(part); + + IgniteCache upstreamCache = ignite.cache(upstreamCacheName); + + ScanQuery qry = new ScanQuery<>(); + qry.setLocal(true); + qry.setPartition(part); + + // TODO: how to guarantee that cache size will not be changed between these calls? + long cnt = upstreamCache.localSizeLong(part); + try (QueryCursor> cursor = upstreamCache.query(qry)) { + return partDataBuilder.build(new UpstreamCursorAdapter<>(cursor.iterator()), cnt, ctx); + } + }); + + return (D)data; + } + + /** + * + * @param ignite + * @param upstreamCacheName + * @param datasetCacheName + * @param ctxBuilder + * @param + * @param + * @param + */ + public static void initContext(Ignite ignite, String upstreamCacheName, + String datasetCacheName, PartitionContextBuilder ctxBuilder, int retries, int interval) { + affinityCallWithRetries(ignite, Arrays.asList(datasetCacheName, upstreamCacheName), part -> { + Ignite locIgnite = Ignition.localIgnite(); + + IgniteCache locUpstreamCache = locIgnite.cache(upstreamCacheName); + + ScanQuery qry = new ScanQuery<>(); + qry.setLocal(true); + qry.setPartition(part); + + long cnt = locUpstreamCache.localSizeLong(part); + C ctx; + try (QueryCursor> cursor = locUpstreamCache.query(qry)) { + ctx = ctxBuilder.build(new UpstreamCursorAdapter<>(cursor.iterator()), cnt); + } + + IgniteCache datasetCache = ignite.cache(datasetCacheName); + + datasetCache.put(part, ctx); + + return part; + }, retries, interval); + } + + /** + * + * @param ignite + * @param upstreamCacheName + * @param datasetCacheName + * @param ctxBuilder + * @param retries + * @param + * @param + * @param + */ + public static void initContext(Ignite ignite, String upstreamCacheName, + String datasetCacheName, PartitionContextBuilder ctxBuilder, int retries) { + initContext(ignite, upstreamCacheName, datasetCacheName, ctxBuilder, retries, 0); + } + + /** + * @param ignite + * @param datasetCacheName + * @param part + * @param + * @return + */ + public static C getContext(Ignite ignite, String datasetCacheName, int part) { + IgniteCache learningCtxCache = ignite.cache(datasetCacheName); + return learningCtxCache.get(part); + } + + /** + * @param ignite + * @param cacheNames + * @param part + */ + private static void checkAllPartitionsAvailable(Ignite ignite, Collection cacheNames, int part) { + for (String cacheName : cacheNames) { + Affinity affinity = ignite.affinity(cacheName); + + ClusterNode partNode = affinity.mapPartitionToNode(part); + ClusterNode locNode = ignite.cluster().localNode(); + + if (!partNode.equals(locNode)) + throw new PartitionNotFoundException(cacheName, locNode.id(), part); + } + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCAffinityFunctionWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/DatasetAffinityFunctionWrapper.java similarity index 92% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCAffinityFunctionWrapper.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/DatasetAffinityFunctionWrapper.java index 2ade520e9e703..0d8c45a04e3d4 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCAffinityFunctionWrapper.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/DatasetAffinityFunctionWrapper.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlc.impl.cache.util; +package org.apache.ignite.ml.dataset.impl.cache.util; import java.util.List; import java.util.UUID; @@ -27,7 +27,7 @@ * Affinity function wrapper which uses key as a partition index and delegates all other functions to specified * delegate. */ -public class DLCAffinityFunctionWrapper implements AffinityFunction { +public class DatasetAffinityFunctionWrapper implements AffinityFunction { /** */ private static final long serialVersionUID = -8233787063079973753L; @@ -39,7 +39,7 @@ public class DLCAffinityFunctionWrapper implements AffinityFunction { * * @param delegate affinity function which actually performs all methods except {@link #partition(Object)} */ - public DLCAffinityFunctionWrapper(AffinityFunction delegate) { + public DatasetAffinityFunctionWrapper(AffinityFunction delegate) { this.delegate = delegate; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCPartitionRecoverableDataStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java similarity index 57% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCPartitionRecoverableDataStorage.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java index 2c50897c9fe49..45282cf91c11e 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCPartitionRecoverableDataStorage.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java @@ -15,32 +15,45 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlc.impl.cache.util; +package org.apache.ignite.ml.dataset.impl.cache.util; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import org.apache.ignite.ml.math.functions.IgniteSupplier; /** - * Partition recoverable data storage is a data structure maintained locally on the node and used to keep recoverable - * part of partitions. * - * @param type of recoverable data */ -public class DLCPartitionRecoverableDataStorage { - /** Map with partitions index as a key and recoverable data as a value. */ - private final ConcurrentMap data = new ConcurrentHashMap<>(); +public class PartitionDataStorage { + /** */ + private final ConcurrentMap storage = new ConcurrentHashMap<>(); - /** Map with partition index as a key and lock as a value. */ + /** */ private final ConcurrentMap locks = new ConcurrentHashMap<>(); - /** */ - public ConcurrentMap getData() { - return data; - } + /** + * + * @param part + * @param supplier + * @return + */ + public Object computeDataIfAbsent(int part, IgniteSupplier supplier) { + Object data = storage.get(part); - /** */ - public ConcurrentMap getLocks() { - return locks; + if (data == null) { + Lock lock = locks.computeIfAbsent(part, p -> new ReentrantLock()); + + lock.lock(); + try { + data = storage.computeIfAbsent(part, p -> supplier.get()); + } + finally { + lock.unlock(); + } + } + + return data; } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/UpstreamPartitionNotFoundException.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionNotFoundException.java similarity index 87% rename from modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/UpstreamPartitionNotFoundException.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionNotFoundException.java index 0734f5b12c944..e5037e8014bc6 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/UpstreamPartitionNotFoundException.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionNotFoundException.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.ml.dlc.impl.cache.util; +package org.apache.ignite.ml.dataset.impl.cache.util; import java.io.Serializable; import java.util.UUID; @@ -26,7 +26,7 @@ * caches with the same affinity function have all their partitions on the same nodes only in terms of eventual * consistency. */ -public class UpstreamPartitionNotFoundException extends IgniteException implements Serializable { +public class PartitionNotFoundException extends IgniteException implements Serializable { /** */ private static final long serialVersionUID = -8891869046312827676L; @@ -40,7 +40,7 @@ public class UpstreamPartitionNotFoundException extends IgniteException implemen * @param nodeId node id * @param partIdx partition index */ - public UpstreamPartitionNotFoundException(String cacheName, UUID nodeId, int partIdx) { + public PartitionNotFoundException(String cacheName, UUID nodeId, int partIdx) { super(String.format(MSG_TEMPLATE, partIdx, cacheName, nodeId.toString())); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java new file mode 100644 index 0000000000000..dba8375f9563b --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java @@ -0,0 +1,58 @@ +/* + * 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.ml.dataset.impl.cache.util; + +import java.util.Iterator; +import javax.cache.Cache; +import org.apache.ignite.ml.dataset.PartitionUpstreamEntry; + +/** + * Cursor adapter used to transform {@link Cache.Entry} received from Ignite Cache query cursor into DLC-specific + * {@link PartitionUpstreamEntry}. + * + * @param type of an upstream value key + * @param type of an upstream value + */ +public class UpstreamCursorAdapter implements Iterator> { + /** Cache entry iterator. */ + private final Iterator> delegate; + + /** + * Constructs a new instance of iterator. + * + * @param delegate cache entry iterator + */ + UpstreamCursorAdapter(Iterator> delegate) { + this.delegate = delegate; + } + + /** {@inheritDoc} */ + @Override public boolean hasNext() { + return delegate.hasNext(); + } + + /** {@inheritDoc} */ + @Override public PartitionUpstreamEntry next() { + Cache.Entry next = delegate.next(); + + if (next == null) + return null; + + return new PartitionUpstreamEntry<>(next.getKey(), next.getValue()); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java new file mode 100644 index 0000000000000..8137251b7546c --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java @@ -0,0 +1,59 @@ +/* + * 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.ml.dataset.impl.local; + +import java.io.Serializable; +import java.util.List; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; +import org.apache.ignite.ml.math.functions.IgniteTriFunction; + +public class LocalDataset implements Dataset { + + private final List ctx; + + private final List data; + + public LocalDataset(List ctx, List data) { + this.ctx = ctx; + this.data = data; + } + + @Override public R computeWithCtx(IgniteTriFunction map, IgniteBinaryOperator reduce, R identity) { + R res = identity; + + for (int part = 0; part < ctx.size(); part++) + res = reduce.apply(res, map.apply(ctx.get(part), data.get(part), part)); + + return res; + } + + @Override public R compute(IgniteBiFunction map, IgniteBinaryOperator reduce, R identity) { + R res = identity; + + for (int part = 0; part < data.size(); part++) + res = reduce.apply(res, map.apply(data.get(part), part)); + + return res; + } + + @Override public void close() { + // Do nothing. + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java new file mode 100644 index 0000000000000..25ee1430e5b53 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java @@ -0,0 +1,107 @@ +/* + * 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.ml.dataset.impl.local; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.dataset.PartitionContextBuilder; +import org.apache.ignite.ml.dataset.PartitionDataBuilder; +import org.apache.ignite.ml.dataset.PartitionUpstreamEntry; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +public class LocalDatasetBuilder + implements DatasetBuilder { + + private final Map upstreamMap; + + private final int partitions; + + private final PartitionContextBuilder partCtxBuilder; + + private final PartitionDataBuilder partDataBuilder; + + public LocalDatasetBuilder(Map upstreamMap, int partitions, + PartitionContextBuilder partCtxBuilder, PartitionDataBuilder partDataBuilder) { + this.upstreamMap = upstreamMap; + this.partitions = partitions; + this.partCtxBuilder = partCtxBuilder; + this.partDataBuilder = partDataBuilder; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public Dataset build() { + List ctx = new ArrayList<>(); + List data = new ArrayList<>(); + + int partSize = upstreamMap.size() / partitions; + + Iterator firstKeysIter = upstreamMap.keySet().iterator(); + Iterator secondKeysIter = upstreamMap.keySet().iterator(); + for (int part = 0; part < partitions; part++) { + int cnt = Math.min((part + 1) * partSize, upstreamMap.size()) - part * partSize; + + C c = partCtxBuilder.build( + new IteratorWindow<>(firstKeysIter, k -> new PartitionUpstreamEntry<>(k, upstreamMap.get(k)), cnt), + cnt + ); + + D d = partDataBuilder.build( + new IteratorWindow<>(secondKeysIter, k -> new PartitionUpstreamEntry<>(k, upstreamMap.get(k)), cnt), + cnt, + c + ); + + ctx.add(c); + data.add(d); + } + + return new LocalDataset<>(ctx, data); + } + + private static class IteratorWindow implements Iterator { + + private final Iterator delegate; + + private final IgniteFunction map; + + private final int cnt; + + private int ptr; + + public IteratorWindow(Iterator delegate, IgniteFunction map, int cnt) { + this.delegate = delegate; + this.map = map; + this.cnt = cnt; + } + + @Override public boolean hasNext() { + return delegate.hasNext() && ptr < cnt; + } + + @Override public T next() { + ptr++; + return map.apply(delegate.next()); + } + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLC.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLC.java deleted file mode 100644 index c6a4d20bc6cc3..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLC.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * 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.ml.dlc; - -import java.io.Serializable; -import org.apache.ignite.ml.math.functions.IgniteBiConsumer; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; -import org.apache.ignite.ml.math.functions.IgniteConsumer; -import org.apache.ignite.ml.math.functions.IgniteFunction; - -/** - * Distributed Learning Context provides the API which allows to perform iterative computation tasks on a distributed - * datasets. Every computation performed via Distributed Learning Context works with {@link DLCPartition} which consists - * of replicated data and recoverable data. Computation task can modify these segments to maintain the iterative - * algorithm context. - * - * @param type of an upstream value key - * @param type of an upstream value - * @param type of replicated data of a partition - * @param type of recoverable data of a partition - */ -public interface DLC extends AutoCloseable { - /** - * Computes a given function on every DLC partition in current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. - * - * @param mapper mapper function applied on every partition - * @param reducer reducer of the results - * @param identity identity value - * @param result type - * @return final reduced result - */ - public R compute(IgniteBiFunction, Integer, R> mapper, IgniteBinaryOperator reducer, - R identity); - - /** - * Computes a given function on every DLC partition in current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. - * - * @param mapper mapper function applied on every partition - * @param reducer reducer of the results - * @param result type - * @return final reduced result - */ - default public R compute(IgniteBiFunction, Integer, R> mapper, - IgniteBinaryOperator reducer) { - return compute(mapper, reducer, null); - } - - /** - * Computes a given function on every DLC partition in current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. - * - * @param mapper mapper function applied on every partition - * @param reducer reducer of the results - * @param identity identity value - * @param result type - * @return final reduced result - */ - default public R compute(IgniteFunction, R> mapper, IgniteBinaryOperator reducer, - R identity) { - return compute((part, partIdx) -> mapper.apply(part), reducer, identity); - } - - /** - * Computes a given function on every DLC partition in current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. - * - * @param mapper mapper function applied on every partition - * @param reducer reducer of the results - * @param result type - * @return final reduced result - */ - default public R compute(IgniteFunction, R> mapper, IgniteBinaryOperator reducer) { - return compute((part, partIdx) -> mapper.apply(part), reducer); - } - - /** - * Computes a given function on every DLC partition in current learning context independently. The goal of this - * approach is to perform {@code mapper} locally on the nodes where partitions are placed and do not involve network - * subsystem where it's possible. - * - * @param mapper mapper function applied on every partition - */ - default public void compute(IgniteBiConsumer, Integer> mapper) { - compute((part, partIdx) -> { - mapper.accept(part, partIdx); - return null; - }, (a, b) -> null); - } - - /** - * Computes a given function on every DLC partition in current learning context independently. The goal of this - * approach is to perform {@code mapper} locally on the nodes where partitions are placed and do not involve network - * subsystem where it's possible. - * - * @param mapper mapper function applied on every partition - */ - default public void compute(IgniteConsumer> mapper) { - compute((part, partIdx) -> mapper.accept(part)); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCFactory.java deleted file mode 100644 index a737b551e42a8..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCFactory.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.ml.dlc; - -import java.io.Serializable; -import org.apache.ignite.ml.math.functions.IgniteFunction; - -/** - * DLC factory which produces distributed learning contexts. - * - * @param type of an upstream value key - * @param type of an upstream value - */ -public interface DLCFactory { - /** - * Constructs a new instance of distributed learning context using specified replicated data transformer, - * recoverable data transformer and DLC wrapper. - * - * @param replicatedTransformer replicated data transformer - * @param recoverableTransformer recoverable data transformer - * @param wrapDLC DLC wrapper - * @param type of replicated data of a partition - * @param type of recoverable data of a partition - * @param type of returned learning context - * @return distributed learning context - */ - public > I createDLC( - DLCPartitionReplicatedTransformer replicatedTransformer, - DLCPartitionRecoverableTransformer recoverableTransformer, - IgniteFunction, I> wrapDLC); - - /** - * Constructs a new instance of distributed learning context using specified replicated data transformer and - * replicated data transformer. - * - * @param replicatedTransformer replicated data transformer - * @param recoverableTransformer recoverable data transformer - * @param type of an upstream value - * @param type of recoverable data of a partition - * @return distributed learning context - */ - default public DLC cteateDLC( - DLCPartitionReplicatedTransformer replicatedTransformer, - DLCPartitionRecoverableTransformer recoverableTransformer) { - return createDLC(replicatedTransformer, recoverableTransformer, e -> e); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartition.java deleted file mode 100644 index 4b1763639ae43..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartition.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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.ml.dlc; - -import java.io.Serializable; - -/** - * Distributed Learning Context partition which consists of replicated data and recoverable data. Replicated - * data is stored in a reliable storage (Ignite Cache) and in case of the node failure or rebalancing automatically - * restored on another node. Recoverable data is stored in a non-reliable local storage and in case of node failure or - * rebalancing when partition is restored on another node should be reloaded from the upstream. - * - * @param type of an upstream value key - * @param type of an upstream value - * @param type of replicated data of a partition - * @param type of recoverable data of a partition - */ -public class DLCPartition implements Serializable { - /** */ - private static final long serialVersionUID = -6348461866724022880L; - - /** Replicated data. */ - private final Q replicatedData; - - /** Loader of the recoverable part of this partition. */ - private final DLCPartitionRecoverableTransformer recoverableDataTransformer; - - /** Recoverable data. */ - private transient W recoverableData; - - /** - * Constructs a new instance of a DLC partition. - * - * @param replicatedData replicated data - * @param recoverableDataTransformer transformer of the recoverable part of this partition - */ - public DLCPartition(Q replicatedData, - DLCPartitionRecoverableTransformer recoverableDataTransformer) { - this.replicatedData = replicatedData; - this.recoverableDataTransformer = recoverableDataTransformer; - } - - /** */ - public Q getReplicatedData() { - return replicatedData; - } - - /** */ - public DLCPartitionRecoverableTransformer getRecoverableDataTransformer() { - return recoverableDataTransformer; - } - - /** */ - public W getRecoverableData() { - return recoverableData; - } - - /** */ - public void setRecoverableData(W recoverableData) { - this.recoverableData = recoverableData; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableTransformer.java deleted file mode 100644 index 1837f86f4f7de..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionRecoverableTransformer.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.ml.dlc; - -import java.io.Serializable; -import org.apache.ignite.ml.math.functions.IgniteFunction; - -/** - * Transformer of the partition recoverable data. - * - * @param type of an upstream value key - * @param type of an upstream value - * @param type of replicated data of a partition - * @param type of recoverable data of a partition - */ -@FunctionalInterface -public interface DLCPartitionRecoverableTransformer - extends Serializable { - /** - * Transforms upstream data to the partition recoverable data. - * - * @param upstreamData upstream data - * @param upstreamDataSize upstream data size - * @param replicatedData replicated data - * @return recoverable data - */ - public W transform(Iterable> upstreamData, Long upstreamDataSize, Q replicatedData); - - /** - * Makes a composition of functions. - * - * @param after function will be called after this one - * @param type of recoverable data of a partition - * @return new transformer - */ - default DLCPartitionRecoverableTransformer andThen( - IgniteFunction after) { - return (upData, upDataSize, replicatedData) -> after.apply(transform(upData, upDataSize, replicatedData)); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedTransformer.java deleted file mode 100644 index 0d01838f2fb04..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/DLCPartitionReplicatedTransformer.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.ml.dlc; - -import java.io.Serializable; -import org.apache.ignite.ml.math.functions.IgniteFunction; - -/** - * Transformer of the partition replicated data. - * - * @param type of an upstream value key - * @param type of an upstream value - * @param type of replicated data of a partition - */ -@FunctionalInterface -public interface DLCPartitionReplicatedTransformer extends Serializable { - /** - * Transforms upstream data to the partition replicated data. - * - * @param upstreamData upstream data - * @param upstreamDataSize upstream data size - * @return replicated data - */ - public Q transform(Iterable> upstreamData, Long upstreamDataSize); - - /** - * Makes a composition of functions. - * - * @param after function will be called after this one - * @param type of replicated data of a partition - * @return new transformer - */ - default DLCPartitionReplicatedTransformer andThen(IgniteFunction after) { - return (upData, upDataSize) -> after.apply(transform(upData, upDataSize)); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCWrapper.java deleted file mode 100644 index bb44d597e89e6..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/DLCWrapper.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.ml.dlc.dataset; - -import java.io.Serializable; -import org.apache.ignite.ml.dlc.DLC; -import org.apache.ignite.ml.dlc.DLCPartition; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; - -/** - * Wrapper of a Distributed Learning Context which allows to introduce new context-specific methods based on base - * {@code compute()} functionality. - * - * @param type of an upstream value key - * @param type of an upstream value - * @param type of replicated data of a partition - * @param type of recoverable data of a partition - */ -public class DLCWrapper implements DLC { - /** Delegate which actually performs base functions like {@code compute()} and {@code close()}. */ - protected final DLC delegate; - - /** - * Constructs a new instance of Distributed Learning Context wrapper - * - * @param delegate delegate which actually performs base functions - */ - public DLCWrapper(DLC delegate) { - this.delegate = delegate; - } - - /** {@inheritDoc} */ - @Override public R compute(IgniteBiFunction, Integer, R> mapper, IgniteBinaryOperator reducer, - R identity) { - return delegate.compute(mapper, reducer, identity); - } - - /** {@inheritDoc} */ - @Override public void close() throws Exception { - delegate.close(); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/DLCTransformers.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/DLCTransformers.java deleted file mode 100644 index 5ae73252b39d1..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/DLCTransformers.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.ml.dlc.dataset.transformer; - -import java.io.Serializable; -import org.apache.ignite.ml.dlc.dataset.DLCDataset; -import org.apache.ignite.ml.dlc.dataset.DLCLabeledDataset; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; - -/** - * Aggregator which allows to find desired transformer from one partition data type to another. This class doesn't - * introduce a new functionality, but helps to work efficiently with existing transformers. - */ -public class DLCTransformers { - /** - * Creates a new instance of transformer which transforms upstream data into the {@link DLCDataset} using the - * specified feature extractor. - * - * @param featureExtractor feature extractor - * @param features number of features - * @param type of an upstream value key - * @param type of an upstream value - * @param type of replicated data of a partition - * @return transformer - */ - public static UpstreamToDatasetTransformer upstreamToDataset( - IgniteBiFunction featureExtractor, int features) { - return new UpstreamToDatasetTransformer<>(featureExtractor, features); - } - - /** - * Creates a new instance of transformer which transforms upstream data into the {@link DLCLabeledDataset} using the - * specified feature and label extractors. - * - * @param featureExtractor feature extractor - * @param lbExtractor label extractor - * @param features number of features - * @param type of an upstream value key - * @param type of an upstream value - * @return transformer - */ - public static UpstreamToLabeledDatasetTransformer upstreamToLabeledDataset( - IgniteBiFunction featureExtractor, - IgniteBiFunction lbExtractor, int features) { - return new UpstreamToLabeledDatasetTransformer<>(featureExtractor, lbExtractor, features); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToDatasetTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToDatasetTransformer.java deleted file mode 100644 index cb441b6cd8b6c..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToDatasetTransformer.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * 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.ml.dlc.dataset.transformer; - -import java.io.Serializable; -import org.apache.ignite.ml.dlc.DLCPartitionRecoverableTransformer; -import org.apache.ignite.ml.dlc.DLCUpstreamEntry; -import org.apache.ignite.ml.dlc.dataset.DLCDataset; -import org.apache.ignite.ml.dlc.dataset.part.DLCDatasetPartitionRecoverable; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; - -/** - * Transforms upstream data into the {@link DLCDataset} using the specified feature extractor. - * - * @param type of an upstream value key - * @param type of an upstream value - */ -public class UpstreamToDatasetTransformer - implements DLCPartitionRecoverableTransformer { - /** */ - private static final long serialVersionUID = -3713681392540367983L; - - /** Feature extractor. */ - private final IgniteBiFunction featureExtractor; - - /** Number of features. */ - private final int features; - - /** - * Constructs a new instance of transformer. - * - * @param featureExtractor feature extractor - * @param features number of features - */ - public UpstreamToDatasetTransformer( - IgniteBiFunction featureExtractor, int features) { - this.featureExtractor = featureExtractor; - this.features = features; - } - - /** - * Transforms upstream data to {@link DLCDatasetPartitionRecoverable}. - * - * @param upstreamData upstream data - * @param upstreamDataSize upstream data size - * @param replicatedData replicated data - * @return dataset partition recoverable data - */ - @Override public DLCDatasetPartitionRecoverable transform(Iterable> upstreamData, Long upstreamDataSize, - Q replicatedData) { - int rows = Math.toIntExact(upstreamDataSize), cols = features; - - double[] features = new double[rows * cols]; - - int ptr = 0; - for (DLCUpstreamEntry e : upstreamData) { - double[] row = featureExtractor.apply(e.getKey(), e.getValue()); - - assert cols == row.length; - - for (int i = 0; i < cols; i++) - features[i * rows + ptr] = row[i]; - - ptr++; - } - - return new DLCDatasetPartitionRecoverable(features, rows, cols); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToLabeledDatasetTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToLabeledDatasetTransformer.java deleted file mode 100644 index e80d448a07f51..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/dataset/transformer/UpstreamToLabeledDatasetTransformer.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * 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.ml.dlc.dataset.transformer; - -import java.io.Serializable; -import org.apache.ignite.ml.dlc.DLCPartitionRecoverableTransformer; -import org.apache.ignite.ml.dlc.DLCUpstreamEntry; -import org.apache.ignite.ml.dlc.dataset.DLCLabeledDataset; -import org.apache.ignite.ml.dlc.dataset.part.DLCLabeledDatasetPartitionRecoverable; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; - -/** - * Transforms upstream data into the {@link DLCLabeledDataset} using the specified feature and label extractors. - * - * @param type of an upstream value key - * @param type of an upstream value - */ -public class UpstreamToLabeledDatasetTransformer - implements DLCPartitionRecoverableTransformer { - /** */ - private static final long serialVersionUID = -1224768715207401297L; - - /** Feature extractor. */ - private final IgniteBiFunction featureExtractor; - - /** Label extractor. */ - private final IgniteBiFunction lbExtractor; - - /** Number of features. */ - private final int features; - - /** - * Constructs a new instance of transformer. - * - * @param featureExtractor feature extractor - * @param lbExtractor label extractor - * @param features number of features - */ - public UpstreamToLabeledDatasetTransformer( - IgniteBiFunction featureExtractor, - IgniteBiFunction lbExtractor, int features) { - this.featureExtractor = featureExtractor; - this.lbExtractor = lbExtractor; - this.features = features; - } - - /** - * Transforms upstream data to {@link DLCLabeledDatasetPartitionRecoverable}. - * - * @param upstreamData upstream data - * @param upstreamDataSize upstream data size - * @param replicatedData replicated data - * @return labeled dataset recoverable data - */ - @Override public DLCLabeledDatasetPartitionRecoverable transform(Iterable> upstreamData, - Long upstreamDataSize, Q replicatedData) { - int rows = Math.toIntExact(upstreamDataSize), cols = features; - - double[] features = new double[rows * cols]; - double[] labels = new double[rows]; - - int ptr = 0; - for (DLCUpstreamEntry e : upstreamData) { - double[] row = featureExtractor.apply(e.getKey(), e.getValue()); - - assert cols == row.length; - - for (int i = 0; i < cols; i++) - features[i * rows + ptr] = row[i]; - - labels[ptr] = lbExtractor.apply(e.getKey(), e.getValue()); - - ptr++; - } - - return new DLCLabeledDatasetPartitionRecoverable(features, rows, cols, labels); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/CacheBasedDLCFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/CacheBasedDLCFactory.java deleted file mode 100644 index 1630bde4f5a8c..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/CacheBasedDLCFactory.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * 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.ml.dlc.impl.cache; - -import java.io.Serializable; -import java.util.Collections; -import java.util.UUID; -import javax.cache.Cache; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.Ignition; -import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.cache.affinity.AffinityFunction; -import org.apache.ignite.cache.query.QueryCursor; -import org.apache.ignite.cache.query.ScanQuery; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.ml.dlc.DLC; -import org.apache.ignite.ml.dlc.DLCFactory; -import org.apache.ignite.ml.dlc.DLCPartition; -import org.apache.ignite.ml.dlc.DLCPartitionRecoverableTransformer; -import org.apache.ignite.ml.dlc.DLCPartitionReplicatedTransformer; -import org.apache.ignite.ml.dlc.impl.cache.util.DLCAffinityFunctionWrapper; -import org.apache.ignite.ml.dlc.impl.cache.util.DLCUpstreamCursorAdapter; -import org.apache.ignite.ml.math.functions.IgniteFunction; - -/** - * DLC factory which produces distributed learning contexts based on the Ignite Cache. - * - * @param type of an upstream value key - * @param type of an upstream value - */ -public class CacheBasedDLCFactory implements DLCFactory { - /** Ignite instance. */ - private final Ignite ignite; - - /** Upstream cache. */ - private final IgniteCache upstreamCache; - - /** - * Constructs a new instance of cache based DLC factory. - * - * @param ignite Ignite instance - * @param upstreamCache upstream cache - */ - public CacheBasedDLCFactory(Ignite ignite, IgniteCache upstreamCache) { - this.ignite = ignite; - this.upstreamCache = upstreamCache; - } - - /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - @Override public > I createDLC( - DLCPartitionReplicatedTransformer replicatedTransformer, - DLCPartitionRecoverableTransformer recoverableTransformer, - IgniteFunction, I> wrapDLC) { - UUID dlcId = UUID.randomUUID(); - - AffinityFunction upstreamCacheAffinity = upstreamCache.getConfiguration(CacheConfiguration.class).getAffinity(); - - CacheConfiguration> dlcCacheCfg = new CacheConfiguration<>(); - dlcCacheCfg.setName(dlcId.toString()); - dlcCacheCfg.setAffinity(new DLCAffinityFunctionWrapper(upstreamCacheAffinity)); - - IgniteCache> dlcCache = ignite.createCache(dlcCacheCfg); - - Affinity affinity = ignite.affinity(dlcCache.getName()); - int partitions = affinity.partitions(); - - for (int partIdx = 0; partIdx < partitions; partIdx++) { - int currPartIdx = partIdx; - - ignite.compute().affinityRun(Collections.singletonList(dlcCache.getName()), partIdx, () -> { - Ignite locIgnite = Ignition.localIgnite(); - IgniteCache locUpstreamCache = locIgnite.cache(upstreamCache.getName()); - - ScanQuery qry = new ScanQuery<>(); - qry.setLocal(true); - qry.setPartition(currPartIdx); - - long cnt = locUpstreamCache.localSizeLong(currPartIdx); - Q replicated; - try (QueryCursor> cursor = locUpstreamCache.query(qry)) { - replicated = replicatedTransformer.transform(new DLCUpstreamCursorAdapter<>(cursor), cnt); - } - DLCPartition part = new DLCPartition<>(replicated, recoverableTransformer); - dlcCache.put(currPartIdx, part); - }); - } - - DLC dlc = new CacheBasedDLCImpl<>(ignite, upstreamCache, dlcCache, dlcId); - - return wrapDLC.apply(dlc); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/CacheBasedDLCImpl.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/CacheBasedDLCImpl.java deleted file mode 100644 index d7c9dfc911e77..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/CacheBasedDLCImpl.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * 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.ml.dlc.impl.cache; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.UUID; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.Ignition; -import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.cluster.ClusterGroup; -import org.apache.ignite.lang.IgniteCallable; -import org.apache.ignite.lang.IgniteFuture; -import org.apache.ignite.ml.dlc.DLC; -import org.apache.ignite.ml.dlc.DLCPartition; -import org.apache.ignite.ml.dlc.impl.cache.util.CacheBasedDLCPartitionBuilder; -import org.apache.ignite.ml.dlc.impl.cache.util.UpstreamPartitionNotFoundException; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; - -/** - * Cache based implementation of a Distributed Learning Context. - * - * @param type of an upstream value key - * @param type of an upstream value - * @param type of replicated data of a partition - * @param type of recoverable data of a partition - */ -public class CacheBasedDLCImpl implements DLC { - /** Ignite instance. */ - private final Ignite ignite; - - /** Upstream cache. */ - private final IgniteCache upstreamCache; - - /** Distributed Learning Context cache. */ - private final IgniteCache> dlcCache; - - /** Distributed Learning Context id. */ - private final UUID dlcId; - - /** - * Constructs a new instance of a cache based distributed learning context. - * - * @param ignite ignite instance - * @param upstreamCache upstream cache - * @param dlcCache distributed learning context cache - * @param dlcId distributed learning context id - */ - public CacheBasedDLCImpl(Ignite ignite, IgniteCache upstreamCache, - IgniteCache> dlcCache, UUID dlcId) { - this.ignite = ignite; - this.upstreamCache = upstreamCache; - this.dlcCache = dlcCache; - this.dlcId = dlcId; - } - - /** {@inheritDoc} */ - @Override public R compute(IgniteBiFunction, Integer, R> mapper, IgniteBinaryOperator reducer, - R identity) { - Affinity affinity = ignite.affinity(dlcCache.getName()); - int partitions = affinity.partitions(); - - String upstreamCacheName = upstreamCache.getName(); - String iddCacheName = dlcCache.getName(); - - Map> calls = new HashMap<>(); - Map> futures = new HashMap<>(); - - ClusterGroup clusterGrp = ignite.cluster().forDataNodes(iddCacheName); - - for (int partIdx = 0; partIdx < partitions; partIdx++) { - final int currPartIdx = partIdx; - - IgniteCallable call = () -> { - CacheBasedDLCPartitionBuilder partBuilder = new CacheBasedDLCPartitionBuilder<>( - Ignition.localIgnite(), - upstreamCacheName, - iddCacheName, - dlcId, - currPartIdx - ); - - DLCPartition part = partBuilder.build(); - - R partRes = mapper.apply(part, currPartIdx); - - IgniteCache> dlcCache = ignite.cache(iddCacheName); - - dlcCache.put(currPartIdx, part); - - return partRes; - }; - - IgniteFuture fut = ignite.compute(clusterGrp).affinityCallAsync( - Arrays.asList(iddCacheName, upstreamCacheName), - currPartIdx, - call - ); - - calls.put(currPartIdx, call); - futures.put(currPartIdx, fut); - } - - R res = identity; - - while (!calls.isEmpty()) { - Iterator>> callIter = calls.entrySet().iterator(); - while (callIter.hasNext()) { - Map.Entry> callEntry = callIter.next(); - - int currPartIdx = callEntry.getKey(); - IgniteCallable call = callEntry.getValue(); - - IgniteFuture fut = futures.get(callEntry.getKey()); - try { - R partRes = fut.get(); - res = reducer.apply(res, partRes); - callIter.remove(); - } - catch (UpstreamPartitionNotFoundException e) { - IgniteFuture newFut = ignite.compute(clusterGrp).affinityCallAsync( - Arrays.asList(iddCacheName, upstreamCacheName), - currPartIdx, - call - ); - futures.put(currPartIdx, newFut); - } - } - } - - return res; - } - - /** {@inheritDoc} */ - @Override public void close() { - ignite.destroyCache(dlcCache.getName()); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/CacheBasedDLCPartitionBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/CacheBasedDLCPartitionBuilder.java deleted file mode 100644 index 3b31859ca0ab4..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/CacheBasedDLCPartitionBuilder.java +++ /dev/null @@ -1,183 +0,0 @@ -/* - * 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.ml.dlc.impl.cache.util; - -import java.io.Serializable; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; -import javax.cache.Cache; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteCluster; -import org.apache.ignite.IgniteCompute; -import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.cache.query.QueryCursor; -import org.apache.ignite.cache.query.ScanQuery; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.lang.IgniteCallable; -import org.apache.ignite.lang.IgniteRunnable; -import org.apache.ignite.ml.dlc.DLCPartition; -import org.apache.ignite.ml.dlc.DLCPartitionRecoverableTransformer; - -/** - * Distributed Learning Context partition builder which constructs a partition from two parts: replicated data which is - * stored in a reliable Ignite Cache and recoverable data which is stored in a local storage and can be recovered from - * the upstream cache. - * - * @param type of an upstream value key - * @param type of an upstream value - * @param type of replicated data of a partition - * @param type of recoverable data of a partition - */ -public class CacheBasedDLCPartitionBuilder { - /** Key used to store local partition storage in the {@link IgniteCluster#nodeLocalMap()}. */ - private static final String NODE_LOCAL_PART_STORAGE_KEY = "ml_local_part_storage"; - - /** Ignite instance. */ - private final Ignite ignite; - - /** Upstream cache name. */ - private final String upstreamCacheName; - - /** Distributed Learning Context cache name. */ - private final String dlcCacheName; - - /** Distributed Learning Context cache name. */ - private final UUID dlcId; - - /** Partition index. */ - private final int partIdx; - - /** - * Constructs a new instance of a cache based Distributed Learning Context partition builder. - * - * @param ignite ignite instance - * @param upstreamCacheName upstream cache name - * @param dlcCacheName distributed learning context cache name - * @param dlcId distributed learning context id - * @param partIdx partition index - */ - public CacheBasedDLCPartitionBuilder(Ignite ignite, String upstreamCacheName, String dlcCacheName, - UUID dlcId, int partIdx) { - this.ignite = ignite; - this.upstreamCacheName = upstreamCacheName; - this.dlcCacheName = dlcCacheName; - this.dlcId = dlcId; - this.partIdx = partIdx; - } - - /** - * Builds a new instance of DLC partition constructed from the replicated and recoverable part. If it's required to - * load recoverable data from the upstream cache but correspondent upstream cache partition is not presented on the - * node the {@link UpstreamPartitionNotFoundException} will be thrown (with assumption that retry can be used in - * this case). - * - * Be aware that this method works correctly only under the condition that partitions of the DLC cache and the - * upstream cache are not moved during the execution. To guarantee this condition please use - * {@link IgniteCompute#affinityCall(String, Object, IgniteCallable)} and - * {@link IgniteCompute#affinityRun(String, Object, IgniteRunnable)} methods or similar to submit the job. - * - * @return distributed learning context partition - */ - public DLCPartition build() { - IgniteCache> dlcCache = ignite.cache(dlcCacheName); - - // Retrieves partition without recoverable data (this data is not stored in DLC cache). - DLCPartition part = dlcCache.get(partIdx); - - DLCPartitionRecoverableDataStorage storage = getLocalPartitionStorage(); - - W recoverableData = storage.getData().get(partIdx); - - // In case when partition just has been moved to the node and haven't been processed here yet recoverable data - // will be empty and we need to load it from the upstream cache. - if (recoverableData == null) { - - // Locks partition loading procedure to avoid multiple memory allocations for the same partition. - Lock partLock = storage.getLocks().computeIfAbsent(partIdx, key -> new ReentrantLock()); - - try { - partLock.lock(); - - // Loads recoverable data from the upstream cache. - recoverableData = storage.getData().computeIfAbsent( - partIdx, - id -> loadRecoverableData(part.getRecoverableDataTransformer(), part.getReplicatedData()) - ); - } - finally { - partLock.unlock(); - } - } - - part.setRecoverableData(recoverableData); - - return part; - } - - /** - * Loads recoverable data from the upstream cache. - * - * @param recoverableDataTransformer recoverable data transformer - * @param replicatedData replicated data - * @return recoverable data - */ - private W loadRecoverableData( - DLCPartitionRecoverableTransformer recoverableDataTransformer, - Q replicatedData) { - Affinity upstreamCacheAffinity = ignite.affinity(upstreamCacheName); - ClusterNode partNode = upstreamCacheAffinity.mapPartitionToNode(partIdx); - - ClusterNode locNode = ignite.cluster().localNode(); - - // If there is not required partition of the upstream cache on the node throws exception. - if (!partNode.equals(locNode)) - throw new UpstreamPartitionNotFoundException(upstreamCacheName, locNode.id(), partIdx); - - IgniteCache locUpstreamCache = ignite.cache(upstreamCacheName); - - ScanQuery qry = new ScanQuery<>(); - qry.setLocal(true); - qry.setPartition(partIdx); - - // TODO: how to guarantee that cache size will not be changed between these calls? - long cnt = locUpstreamCache.localSizeLong(partIdx); - try (QueryCursor> cursor = locUpstreamCache.query(qry)) { - return recoverableDataTransformer.transform(new DLCUpstreamCursorAdapter<>(cursor), cnt, replicatedData); - } - } - - /** - * Retrieves partition recoverable data storage from the {@link IgniteCluster#nodeLocalMap()}. - * - * @return partition recoverable storage - */ - @SuppressWarnings("unchecked") - private DLCPartitionRecoverableDataStorage getLocalPartitionStorage() { - ConcurrentMap nodeLocMap = ignite.cluster().nodeLocalMap(); - - ConcurrentMap> locPartStorage = - (ConcurrentMap>) - nodeLocMap.computeIfAbsent(NODE_LOCAL_PART_STORAGE_KEY, key -> new ConcurrentHashMap<>()); - - return locPartStorage.computeIfAbsent(dlcId, key -> new DLCPartitionRecoverableDataStorage<>()); - } -} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCUpstreamCursorAdapter.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCUpstreamCursorAdapter.java deleted file mode 100644 index 8bb0fc8607c47..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/cache/util/DLCUpstreamCursorAdapter.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.ml.dlc.impl.cache.util; - -import java.util.Iterator; -import javax.cache.Cache; -import org.apache.ignite.cache.query.QueryCursor; -import org.apache.ignite.ml.dlc.DLCUpstreamEntry; - -/** - * Cursor adapter used to transform {@link Cache.Entry} received from Ignite Cache query cursor into DLC-specific - * {@link DLCUpstreamEntry}. - * - * @param type of an upstream value key - * @param type of an upstream value - */ -public class DLCUpstreamCursorAdapter implements Iterable> { - /** Ignite Cache cursor. */ - private final QueryCursor> cursor; - - /** - * Constructs a new instance of query cursor adapter. - * - * @param cursor Ignite Cache query cursor - */ - public DLCUpstreamCursorAdapter(QueryCursor> cursor) { - this.cursor = cursor; - } - - /** {@inheritDoc} */ - @Override public Iterator> iterator() { - return new DLCUpstreamCursorWrapperIterator(cursor.iterator()); - } - - /** - * Utils class representing iterator of {@link DLCUpstreamEntry} based on iterator of {@link Cache.Entry}. - */ - private final class DLCUpstreamCursorWrapperIterator implements Iterator> { - /** Cache entry iterator. */ - private final Iterator> delegate; - - /** - * Constructs a new instance of iterator. - * - * @param delegate cache entry iterator - */ - DLCUpstreamCursorWrapperIterator(Iterator> delegate) { - this.delegate = delegate; - } - - /** {@inheritDoc} */ - @Override public boolean hasNext() { - return delegate.hasNext(); - } - - /** {@inheritDoc} */ - @Override public DLCUpstreamEntry next() { - Cache.Entry next = delegate.next(); - - if (next == null) - return null; - - return new DLCUpstreamEntry<>(next.getKey(), next.getValue()); - } - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCFactory.java deleted file mode 100644 index 2d2c54370c414..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCFactory.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * 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.ml.dlc.impl.local; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.ignite.ml.dlc.DLC; -import org.apache.ignite.ml.dlc.DLCFactory; -import org.apache.ignite.ml.dlc.DLCPartition; -import org.apache.ignite.ml.dlc.DLCPartitionRecoverableTransformer; -import org.apache.ignite.ml.dlc.DLCPartitionReplicatedTransformer; -import org.apache.ignite.ml.dlc.impl.local.util.DLCUpstreamMapAdapter; -import org.apache.ignite.ml.math.functions.IgniteFunction; - -/** - * DLC factory which produces distributed learning contexts based on the local {@code Map}. - * - * @param type of an upstream value key - * @param type of an upstream value - */ -public class MapBasedDLCFactory implements DLCFactory { - /** Upstream data. */ - private final Map upstreamData; - - /** Number of partitions. */ - private final int partitions; - - /** - * Constructs a new instance of {@code Map} based DLC factory. - * - * @param upstreamData upstream data - * @param partitions partitions - */ - public MapBasedDLCFactory(Map upstreamData, int partitions) { - this.upstreamData = upstreamData; - this.partitions = partitions; - } - - /** {@inheritDoc} */ - @Override public > I createDLC( - DLCPartitionReplicatedTransformer replicatedTransformer, - DLCPartitionRecoverableTransformer recoverableTransformer, - IgniteFunction, I> wrapDLC) { - Map> dlcMap = new HashMap<>(); - - int partSize = upstreamData.size() / partitions; - - List keys = new ArrayList<>(upstreamData.keySet()); - - for (int partIdx = 0; partIdx < partitions; partIdx++) { - List partKeys = keys.subList(partIdx * partSize, Math.min((partIdx + 1) * partSize, upstreamData.size())); - Q replicated = replicatedTransformer.transform( - new DLCUpstreamMapAdapter<>(upstreamData, partKeys), - (long) partKeys.size() - ); - W recoverable = recoverableTransformer.transform( - new DLCUpstreamMapAdapter<>(upstreamData, partKeys), - (long) partKeys.size(), - replicated - ); - DLCPartition part = new DLCPartition<>(replicated, null); - part.setRecoverableData(recoverable); - dlcMap.put(partIdx, part); - } - - DLC dlc = new MapBasedDLCImpl<>(dlcMap, partitions); - - return wrapDLC.apply(dlc); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCImpl.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCImpl.java deleted file mode 100644 index 11a82dc8105f8..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/MapBasedDLCImpl.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * 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.ml.dlc.impl.local; - -import java.io.Serializable; -import java.util.Iterator; -import java.util.Map; -import javax.cache.Cache; -import org.apache.ignite.ml.dlc.DLC; -import org.apache.ignite.ml.dlc.DLCPartition; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; -import org.jetbrains.annotations.NotNull; - -/** - * Local Map based implementation of a Distributed Learning Context. - * - * @param type of an upstream value key - * @param type of an upstream value - * @param type of replicated data of a partition - * @param type of recoverable data of a partition - */ -public class MapBasedDLCImpl - implements DLC { - /** Map containing pairs of partition index and partitions. */ - private final Map> dlcMap; - - /** Number of partitions. */ - private final int partitions; - - /** - * Constructs a new instance of a local Map based Distributed Learning Context. - * - * @param dlcMap distributed learning context map - * @param partitions number of partitions - */ - public MapBasedDLCImpl(Map> dlcMap, int partitions) { - this.dlcMap = dlcMap; - this.partitions = partitions; - } - - /** {@inheritDoc} */ - @Override public R compute(IgniteBiFunction, Integer, R> mapper, IgniteBinaryOperator reducer, - R identity) { - - R res = identity; - - for (int partIdx = 0; partIdx < partitions; partIdx++) { - DLCPartition part = dlcMap.get(partIdx); - - R partRes = mapper.apply(part, partIdx); - - res = reducer.apply(res, partRes); - - dlcMap.put(partIdx, part); - } - - return res; - } - - /** {@inheritDoc} */ - @Override public void close() { - dlcMap.clear(); - } - - private class MapBasedCacheEntry implements Cache.Entry { - - private final K key; - - private final V value; - - public MapBasedCacheEntry(K key, V value) { - this.key = key; - this.value = value; - } - - @Override public K getKey() { - return key; - } - - @Override public V getValue() { - return value; - } - - @Override public T unwrap(Class clazz) { - throw new UnsupportedOperationException(); - } - } - - private class MapBasedIterable implements Iterable> { - - private final Iterator> iterator; - - public MapBasedIterable(Iterator> iterator) { - this.iterator = iterator; - } - - @NotNull @Override public Iterator> iterator() { - return iterator; - } - } -} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/util/DLCUpstreamMapAdapter.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/util/DLCUpstreamMapAdapter.java deleted file mode 100644 index b3622058db1b7..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlc/impl/local/util/DLCUpstreamMapAdapter.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * 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.ml.dlc.impl.local.util; - -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import org.apache.ignite.ml.dlc.DLCUpstreamEntry; - -/** - * This is {@code Map} adapter used to transform {@code Map} entries into DLC-specific {@link DLCUpstreamEntry}. - * - * @param type of an upstream value key - * @param type of an upstream value - */ -public class DLCUpstreamMapAdapter implements Iterable> { - /** Upstream data {@code Map}. */ - private final Map upstreamData; - - /** Ordered upstream data keys. */ - private final List keys; - - /** - * Constructs a new instance of {@code Map} adapter. - * - * @param upstreamData upstream data {@code Map} - * @param keys ordered upstream data keys - */ - public DLCUpstreamMapAdapter(Map upstreamData, List keys) { - this.upstreamData = upstreamData; - this.keys = keys; - } - - /** {@inheritDoc} */ - @Override public Iterator> iterator() { - return new DLCUpstreamKeysWrapperIterator(keys.iterator()); - } - - /** - * Utils class representing iterator of {@link DLCUpstreamEntry} based on iterator of {@code Map} keys. - */ - private final class DLCUpstreamKeysWrapperIterator implements Iterator> { - /** Keys iterator. */ - private final Iterator keysIterator; - - /** - * Constructs a new instance of iterator. - * - * @param keysIterator keys iterator - */ - DLCUpstreamKeysWrapperIterator(Iterator keysIterator) { - this.keysIterator = keysIterator; - } - - /** {@inheritDoc} */ - @Override public boolean hasNext() { - return keysIterator.hasNext(); - } - - /** {@inheritDoc} */ - @Override public DLCUpstreamEntry next() { - K nextKey = keysIterator.next(); - - if (nextKey == null) - return null; - - return new DLCUpstreamEntry<>(nextKey, upstreamData.get(nextKey)); - } - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/ColumnDecisionTreeTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/ColumnDecisionTreeTrainer.java index fec0a83fd503c..ef703846026da 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/ColumnDecisionTreeTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/ColumnDecisionTreeTrainer.java @@ -172,7 +172,7 @@ private static class IndexAndSplitInfo { } /** - * Utility class used to build decision tree. Basically it is pointer to leaf node. + * Utility class used to withDataBuilder decision tree. Basically it is pointer to leaf node. */ private static class TreeTip { /** */ diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java b/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java index f6cf2d556f1fe..766417b72d7d4 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java @@ -6,9 +6,8 @@ import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.dlc.dataset.DLCDataset; -import org.apache.ignite.ml.dlc.dataset.transformer.DLCTransformers; -import org.apache.ignite.ml.dlc.impl.cache.CacheBasedDLCFactory; +import org.apache.ignite.ml.dataset.DatasetFactory; +import org.apache.ignite.ml.dataset.api.SimpleDataset; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; /** */ @@ -49,12 +48,7 @@ public void testTest() { for (int i = 0; i < 40; i++) cache.put(i, i); - DLCDataset dataset = new CacheBasedDLCFactory<>(ignite, cache) - .createDLC( - (data, size) -> null, - DLCTransformers.upstreamToDataset((k, v) -> new double[]{1, 2, 3}, 3), - DLCDataset::new - ); + SimpleDataset dataset = DatasetFactory.createSimpleDataset(ignite, cache, (k, v) -> new double[] { 42.0 }, 1); // Calculation of the mean value. This calculation will be performed in map-reduce manner. double[] mean = dataset.mean(); From f83153e16b1dba9eed6c6e26b717726f8b416c96 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Sat, 27 Jan 2018 18:15:06 +0300 Subject: [PATCH 19/38] IGNITE-7437 Updated javadoc and tests, removed d-learn part. --- .../org/apache/ignite/ml/dataset/Dataset.java | 173 ++++++------ .../ignite/ml/dataset/DatasetBuilder.java | 17 +- .../ignite/ml/dataset/DatasetFactory.java | 256 ++++++++++------- .../ml/dataset/PartitionContextBuilder.java | 31 +- .../ml/dataset/PartitionDataBuilder.java | 36 ++- ...nUpstreamEntry.java => UpstreamEntry.java} | 10 +- .../ignite/ml/dataset/api/DatasetWrapper.java | 19 +- .../ignite/ml/dataset/api/SimpleDataset.java | 8 +- .../ml/dataset/api/SimpleLabeledDataset.java | 12 +- .../builder/context/EmptyContextBuilder.java | 13 +- .../data/SimpleDatasetDataBuilder.java | 31 +- .../data/SimpleLabeledDatasetDataBuilder.java | 30 +- .../ml/dataset/api/context/EmptyContext.java | 5 + .../dataset/api/data/SimpleDatasetData.java | 11 +- .../api/data/SimpleLabeledDatasetData.java | 19 +- .../dataset/impl/cache/CacheBasedDataset.java | 59 ++-- .../impl/cache/CacheBasedDatasetBuilder.java | 32 ++- .../dataset/impl/cache/util/ComputeUtils.java | 129 +++++---- .../util/DatasetAffinityFunctionWrapper.java | 8 +- .../impl/cache/util/PartitionDataStorage.java | 22 +- .../util/PartitionNotFoundException.java | 12 +- .../cache/util/UpstreamCursorAdapter.java | 10 +- .../ml/dataset/impl/local/LocalDataset.java | 34 ++- .../impl/local/LocalDatasetBuilder.java | 74 +++-- .../ignite/ml/dlearn/DLearnContext.java | 128 --------- .../ml/dlearn/DLearnContextFactory.java | 35 --- .../ml/dlearn/DLearnPartitionFactory.java | 38 --- .../ml/dlearn/DLearnPartitionStorage.java | 49 ---- .../context/cache/CacheDLearnContext.java | 202 ------------- .../cache/CacheDLearnContextFactory.java | 122 -------- .../context/cache/CacheDLearnPartition.java | 75 ----- .../cache/CacheDLearnPartitionStorage.java | 113 -------- .../ml/dlearn/context/cache/package-info.java | 22 -- .../DLearnPartitionAffinityFunction.java | 112 -------- .../context/local/LocalDLearnContext.java | 132 --------- .../local/LocalDLearnContextFactory.java | 80 ------ .../context/local/LocalDLearnPartition.java | 62 ---- .../local/LocalDLearnPartitionStorage.java | 68 ----- .../ml/dlearn/context/local/package-info.java | 22 -- .../ml/dlearn/context/package-info.java | 22 -- .../transformer/DLearnContextTransformer.java | 34 --- .../DLearnContextTransformers.java | 77 ----- ...acheDatasetDLearnPartitionTransformer.java | 129 --------- ...eledDatasetDLearnPartitionTransformer.java | 138 --------- .../transformer/cache/package-info.java | 22 -- ...ocalDatasetDLearnPartitionTransformer.java | 97 ------- ...eledDatasetDLearnPartitionTransformer.java | 114 -------- .../transformer/local/package-info.java | 22 -- .../context/transformer/package-info.java | 22 -- .../dataset/AbstractDLearnContextWrapper.java | 68 ----- .../ml/dlearn/dataset/DLearnDataset.java | 206 -------------- .../dlearn/dataset/DLearnLabeledDataset.java | 34 --- .../ml/dlearn/dataset/package-info.java | 22 -- .../dataset/part/DLeanDatasetPartition.java | 87 ------ .../part/DLearnLabeledDatasetPartition.java | 68 ----- .../ml/dlearn/dataset/part/package-info.java | 22 -- .../apache/ignite/ml/dlearn/package-info.java | 22 -- .../utils/DLearnContextPartitionKey.java | 93 ------ .../ignite/ml/dlearn/utils/package-info.java | 22 -- .../linear/LinearRegressionLSQRTrainer.java | 84 ------ .../ml/dataset/api/DatasetWrapperTest.java | 4 + .../cache/CacheBasedDatasetBuilderTest.java | 95 +++++++ .../impl/cache/CacheBasedDatasetTest.java} | 266 +++--------------- .../impl/local/LocalDatasetBuilderTest.java | 4 + .../dataset/impl/local/LocalDatasetTest.java | 4 + .../cache/CacheDLearnContextFactoryTest.java | 118 -------- .../DLearnPartitionAffinityFunctionTest.java | 24 -- .../context/local/LocalDLearnContextTest.java | 125 -------- 68 files changed, 840 insertions(+), 3516 deletions(-) rename modules/ml/src/main/java/org/apache/ignite/ml/dataset/{PartitionUpstreamEntry.java => UpstreamEntry.java} (84%) delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContextFactory.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionFactory.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionStorage.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartitionStorage.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunction.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartitionStorage.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnDataset.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnLabeledDataset.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLeanDatasetPartition.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLearnLabeledDatasetPartition.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextPartitionKey.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dataset/api/DatasetWrapperTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilderTest.java rename modules/ml/src/test/java/org/apache/ignite/ml/{dlearn/context/cache/CacheDLearnContextTest.java => dataset/impl/cache/CacheBasedDatasetTest.java} (52%) create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilderTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactoryTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunctionTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextTest.java diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/Dataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/Dataset.java index a3c40947abb28..4471cd900875c 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/Dataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/Dataset.java @@ -18,6 +18,8 @@ package org.apache.ignite.ml.dataset; import java.io.Serializable; +import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDataset; +import org.apache.ignite.ml.dataset.impl.local.LocalDataset; import org.apache.ignite.ml.math.functions.IgniteBiConsumer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; @@ -27,133 +29,138 @@ import org.apache.ignite.ml.math.functions.IgniteTriFunction; /** - * Distributed Learning Context provides the API which allows to perform iterative computation tasks on a distributed - * datasets. Every computation performed via Distributed Learning Context works with {@link DLCPartition} which consists - * of replicated data and recoverable data. Computation task can modify these segments to maintain the iterative - * algorithm context. + * A dataset providing an API that allows to perform generic computations on a distributed data represented as a set of + * partitions distributed across a cluster or placed locally. Every partition contains a {@code context} (reliably + * stored segment) and {@code data} (unreliably stored segment, which can be recovered from an upstream data and a + * {@code context} if needed). Computations are performed in a {@code MapReduce} manner, what allows to reduce a + * network traffic for most of the machine learning algorithms. * - * @param - * @param + *

Dataset functionality allows to implement iterative machine learning algorithms via introducing computation + * context. In case iterative algorithm requires to maintain a state available and updatable on every iteration this + * state can be stored in the {@code context} of the partition and after that it will be available in further + * computations even if the Ignite Cache partition will be moved to another node because of node failure or rebalancing. + * + *

Partition {@code context} should be {@link Serializable} to be saved in Ignite Cache. Partition {@code data} + * should be {@link AutoCloseable} to allow system to clean up correspondent resources when partition {@code data} is + * not needed anymore. + * + * @param type of a partition {@code context} + * @param type of a partition {@code data} + * + * @see CacheBasedDataset + * @see LocalDataset + * @see DatasetFactory */ public interface Dataset extends AutoCloseable { /** - * Computes the given function on every partition in the current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. + * Applies the specified {@code map} function to every partition {@code data}, {@code context} and partition + * index in the dataset and then reduces {@code map} results to final result by using the {@code reduce} function. * - * @param map - * @param reduce - * @param identity - * @param - * @return + * @param map function applied to every partition {@code data}, {@code context} and partition index + * @param reduce function applied to results of {@code map} to get final result + * @param identity identity + * @param type of a result + * @return final result */ public R computeWithCtx(IgniteTriFunction map, IgniteBinaryOperator reduce, R identity); /** - * Computes the given function on every partition in the current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. + * Applies the specified {@code map} function to every partition {@code data} and partition index in the dataset + * and then reduces {@code map} results to final result by using the {@code reduce} function. * - * @param map - * @param reduce - * @param identity - * @param - * @return + * @param map function applied to every partition {@code data} and partition index + * @param reduce function applied to results of {@code map} to get final result + * @param identity identity + * @param type of a result + * @return final result */ public R compute(IgniteBiFunction map, IgniteBinaryOperator reduce, R identity); /** - * Computes the given function on every partition in the current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. + * Applies the specified {@code map} function to every partition {@code data}, {@code context} and partition + * index in the dataset and then reduces {@code map} results to final result by using the {@code reduce} function. * - * @param map - * @param reduce - * @param - * @return + * @param map function applied to every partition {@code data}, {@code context} and partition index + * @param reduce function applied to results of {@code map} to get final result + * @param type of a result + * @return final result */ default public R computeWithCtx(IgniteTriFunction map, IgniteBinaryOperator reduce) { return computeWithCtx(map, reduce, null); } /** - * Computes the given function on every partition in the current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. + * Applies the specified {@code map} function to every partition {@code data} and partition index in the dataset + * and then reduces {@code map} results to final result by using the {@code reduce} function. * - * @param map - * @param reduce - * @param - * @return + * @param map function applied to every partition {@code data} and partition index + * @param reduce function applied to results of {@code map} to get final result + * @param type of a result + * @return final result */ default public R compute(IgniteBiFunction map, IgniteBinaryOperator reduce) { return compute(map, reduce, null); } /** - * Computes the given function on every partition in the current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. + * Applies the specified {@code map} function to every partition {@code data} and {@code context} in the dataset + * and then reduces {@code map} results to final result by using the {@code reduce} function. * - * @param map - * @param reduce - * @param identity - * @param - * @return + * @param map function applied to every partition {@code data} and {@code context} + * @param reduce function applied to results of {@code map} to get final result + * @param identity identity + * @param type of a result + * @return final result */ default public R computeWithCtx(IgniteBiFunction map, IgniteBinaryOperator reduce, R identity) { return computeWithCtx((ctx, data, partIdx) -> map.apply(ctx, data), reduce, identity); } /** - * Computes the given function on every partition in the current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. + * Applies the specified {@code map} function to every partition {@code data} in the dataset and then reduces + * {@code map} results to final result by using the {@code reduce} function. * - * @param map - * @param reduce - * @param identity - * @param - * @return + * @param map function applied to every partition {@code data} + * @param reduce function applied to results of {@code map} to get final result + * @param identity identity + * @param type of a result + * @return final result */ default public R compute(IgniteFunction map, IgniteBinaryOperator reduce, R identity) { return compute((data, partIdx) -> map.apply(data), reduce, identity); } /** - * Computes the given function on every partition in the current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. + * Applies the specified {@code map} function to every partition {@code data} and {@code context} in the dataset + * and then reduces {@code map} results to final result by using the {@code reduce} function. * - * @param map - * @param reduce - * @param - * @return + * @param map function applied to every partition {@code data} and {@code context} + * @param reduce function applied to results of {@code map} to get final result + * @param type of a result + * @return final result */ default public R computeWithCtx(IgniteBiFunction map, IgniteBinaryOperator reduce) { return computeWithCtx((ctx, data, partIdx) -> map.apply(ctx, data), reduce); } /** - * Computes the given function on every partition in the current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. + * Applies the specified {@code map} function to every partition {@code data} in the dataset and then reduces + * {@code map} results to final result by using the {@code reduce} function. * - * @param map - * @param reduce - * @param - * @return + * @param map function applied to every partition {@code data} + * @param reduce function applied to results of {@code map} to get final result + * @param type of a result + * @return final result */ default public R compute(IgniteFunction map, IgniteBinaryOperator reduce) { return compute((data, partIdx) -> map.apply(data), reduce); } /** - * Computes the given function on every partition in the current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. + * Applies the specified {@code map} function to every partition {@code data}, {@code context} and partition + * index in the dataset. * - * @param map + * @param map function applied to every partition {@code data}, {@code context} and partition index */ default public void computeWithCtx(IgniteTriConsumer map) { computeWithCtx((ctx, data, partIdx) -> { @@ -163,11 +170,9 @@ default public void computeWithCtx(IgniteTriConsumer map) { } /** - * Computes the given function on every partition in the current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. + * Applies the specified {@code map} function to every partition {@code data} in the dataset and partition index. * - * @param map + * @param map function applied to every partition {@code data} and partition index */ default public void compute(IgniteBiConsumer map) { compute((data, partIdx) -> { @@ -177,32 +182,30 @@ default public void compute(IgniteBiConsumer map) { } /** - * Computes the given function on every partition in the current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. + * Applies the specified {@code map} function to every partition {@code data} and {@code context} in the dataset. * - * @param map + * @param map function applied to every partition {@code data} and {@code context} */ default public void computeWithCtx(IgniteBiConsumer map) { computeWithCtx((ctx, data, partIdx) -> map.accept(ctx, data)); } /** - * Computes the given function on every partition in the current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code map} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. + * Applies the specified {@code map} function to every partition {@code data} in the dataset. * - * @param map + * @param map function applied to every partition {@code data} */ default public void compute(IgniteConsumer map) { compute((data, partIdx) -> map.accept(data)); } /** + * Wraps this dataset into the specified wrapper to introduce new functionality based on {@code compute} and + * {@code computeWithCtx} methods. * - * @param wrapper - * @param - * @return + * @param wrapper dataset wrapper + * @param type of a new wrapped dataset + * @return new wrapped dataset */ default public > I wrap(IgniteFunction, I> wrapper) { return wrapper.apply(this); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java index 102e372217650..4bc082e5e981a 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java @@ -18,16 +18,27 @@ package org.apache.ignite.ml.dataset; import java.io.Serializable; +import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; /** + * A builder constructing instances of a {@link Dataset}. Implementations of this interface encapsulate logic of + * building specific datasets such as allocation required data structures and initialization of {@code context} part of + * partitions. * - * @param - * @param + * @param type of a partition {@code context} + * @param type of a partition {@code data} + * + * @see CacheBasedDatasetBuilder + * @see LocalDatasetBuilder + * @see Dataset */ public interface DatasetBuilder { /** + * Constructs a new instance of {@link Dataset} that includes allocation required data structures and + * initialization of {@code context} part of partitions. * - * @return + * @return dataset */ public Dataset build(); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetFactory.java index 2bf5336f65c0b..32ece9478ba85 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetFactory.java @@ -27,23 +27,63 @@ import org.apache.ignite.ml.dataset.api.builder.data.SimpleDatasetDataBuilder; import org.apache.ignite.ml.dataset.api.builder.data.SimpleLabeledDatasetDataBuilder; import org.apache.ignite.ml.dataset.api.context.EmptyContext; +import org.apache.ignite.ml.dataset.api.data.SimpleDatasetData; +import org.apache.ignite.ml.dataset.api.data.SimpleLabeledDatasetData; import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.apache.ignite.ml.math.functions.IgniteBiFunction; +/** + * Factory providing a client facing API that allows to construct basic and the most frequently used types of dataset. + * + * + *

Dataset construction is based on three major concepts: a partition {@code upstream}, {@code context} and + * {@code data}. A partition {@code upstream} is a data source, which assumed to be available all the time regardless + * node failures and rebalancing events. A partition {@code context} is a part of a partition maintained during the + * whole computation process and stored in a reliable storage so that a {@code context} is staying available and + * consistent regardless node failures and rebalancing events as well as an {@code upstream}. A partition {@code data} + * is a part of partition maintained during a computation process in unreliable local storage such as heap, off-heap or + * GPU memory on the node where current computation is performed, so that partition {@code data} can be lost as result + * of node failure or rebalancing, but it can be restored from an {@code upstream} and a partition {@code context}. + * + *

A partition {@code context} and {@code data} are built on top of an {@code upstream} by using specified + * builders: {@link PartitionContextBuilder} and {@link PartitionDataBuilder} correspondingly. To build a generic + * dataset the following approach is used: + * + * + * {@code + * Dataset dataset = DatasetFactory.create( + * ignite, + * cache, + * partitionContextBuilder, + * partitionDataBuilder + * ); + * } + * + * + *

As well as the generic building method {@code create} this factory provides methods that allow to create a + * specific dataset types such as method {@code createSimpleDataset} to create {@link SimpleDataset} and method + * {@code createSimpleLabeledDataset} to create {@link SimpleLabeledDataset}. + * + * @see Dataset + * @see PartitionContextBuilder + * @see PartitionDataBuilder + */ public class DatasetFactory { /** + * Creates a new instance of distributed dataset using the specified {@code partCtxBuilder} and + * {@code partDataBuilder}. This is the generic methods that allows to create any Ignite Cache based datasets with + * any desired partition {@code context} and {@code data}. * - * - * @param ignite - * @param upstreamCache - * @param partCtxBuilder - * @param partDataBuilder - * @param - * @param - * @param - * @param - * @return + * @param ignite Ignite instance + * @param upstreamCache Ignite Cache with {@code upstream} data + * @param partCtxBuilder partition {@code context} builder + * @param partDataBuilder partition {@code data} builder + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @param type of a partition {@code context} + * @param type of a partition {@code data} + * @return dataset */ public static Dataset create( Ignite ignite, IgniteCache upstreamCache, PartitionContextBuilder partCtxBuilder, @@ -52,17 +92,19 @@ public static Dataset - * @param - * @param - * @return + * @param ignite Ignite instance + * @param upstreamCache Ignite Cache with {@code upstream} data + * @param partCtxBuilder partition {@code context} builder + * @param featureExtractor feature extractor used to extract features and build {@link SimpleDatasetData} + * @param cols number of columns (features) will be extracted + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @param type of a partition {@code context} + * @return dataset */ public static SimpleDataset createSimpleDataset(Ignite ignite, IgniteCache upstreamCache, PartitionContextBuilder partCtxBuilder, @@ -76,18 +118,20 @@ public static SimpleDataset createSimpleDatase } /** + * Creates a new instance of distributed {@link SimpleLabeledDataset} using the specified {@code partCtxBuilder}, + * {@code featureExtractor} and {@code lbExtractor}. This method determines partition {@code data} to be + * {@link SimpleLabeledDatasetData}, but allows to use any desired type of partition {@code context}. * - * - * @param ignite - * @param upstreamCache - * @param partCtxBuilder - * @param featureExtractor - * @param lbExtractor - * @param cols - * @param - * @param - * @param - * @return + * @param ignite Ignite instance + * @param upstreamCache Ignite Cache with {@code upstream} data + * @param partCtxBuilder partition {@code context} builder + * @param featureExtractor feature extractor used to extract features and build {@link SimpleLabeledDatasetData} + * @param lbExtractor label extractor used to extract labels and buikd {@link SimpleLabeledDatasetData} + * @param cols number of columns (features) will be extracted + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @param type of a partition {@code context} + * @return dataset */ public static SimpleLabeledDataset createSimpleLabeledDataset(Ignite ignite, IgniteCache upstreamCache, PartitionContextBuilder partCtxBuilder, @@ -101,15 +145,17 @@ public static SimpleLabeledDataset createSimpl } /** + * Creates a new instance of distributed {@link SimpleDataset} using the specified {@code featureExtractor}. This + * methods determines partition {@code context} to be {@link EmptyContext} and partition {@code data} to be + * {@link SimpleDatasetData}. * - * - * @param ignite - * @param upstreamCache - * @param featureExtractor - * @param cols - * @param - * @param - * @return + * @param ignite Ignite instance + * @param upstreamCache Ignite Cache with {@code upstream} data + * @param featureExtractor feature extractor used to extract features and build {@link SimpleDatasetData} + * @param cols number of columns (features) will be extracted + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @return dataset */ public static SimpleDataset createSimpleDataset(Ignite ignite, IgniteCache upstreamCache, IgniteBiFunction featureExtractor, int cols) { @@ -117,35 +163,40 @@ public static SimpleDataset createSimpleDataset(Ignite igni } /** + * Creates a new instance of distributed {@link SimpleLabeledDataset} using the specified {@code featureExtractor} + * and {@code lbExtractor}. This methods determines partition {@code context} to be {@link EmptyContext} and + * partition {@code data} to be {@link SimpleLabeledDatasetData}. * - * - * @param ignite - * @param upstreamCache - * @param featureExtractor - * @param lbExtractor - * @param cols - * @param - * @param - * @return + * @param ignite Ignite instance + * @param upstreamCache Ignite Cache with {@code upstream} data + * @param featureExtractor feature extractor used to extract features and build {@link SimpleLabeledDatasetData} + * @param lbExtractor label extractor used to extract labels and buikd {@link SimpleLabeledDatasetData} + * @param cols number of columns (features) will be extracted + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @return dataset */ public static SimpleLabeledDataset createSimpleLabeledDataset(Ignite ignite, IgniteCache upstreamCache, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor, int cols) { - return createSimpleLabeledDataset(ignite, upstreamCache, new EmptyContextBuilder<>(), featureExtractor, lbExtractor, cols); + return createSimpleLabeledDataset(ignite, upstreamCache, new EmptyContextBuilder<>(), featureExtractor, + lbExtractor, cols); } /** + * Creates a new instance of local dataset using the specified {@code partCtxBuilder} and {@code partDataBuilder}. + * This is the generic methods that allows to create any Ignite Cache based datasets with any desired partition + * {@code context} and {@code data}. * - * - * @param upstreamMap - * @param partitions - * @param partCtxBuilder - * @param partDataBuilder - * @param - * @param - * @param - * @param - * @return + * @param upstreamMap {@code Map} with {@code upstream} data + * @param partitions number of partitions {@code upstream} {@code Map} will be divided on + * @param partCtxBuilder partition {@code context} builder + * @param partDataBuilder partition {@code data} builder + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @param type of a partition {@code context} + * @param type of a partition {@code data} + * @return dataset */ public static Dataset create( Map upstreamMap, int partitions, PartitionContextBuilder partCtxBuilder, @@ -154,17 +205,19 @@ public static Dataset - * @param - * @param - * @return + * @param upstreamMap {@code Map} with {@code upstream} data + * @param partitions number of partitions {@code upstream} {@code Map} will be divided on + * @param partCtxBuilder partition {@code context} builder + * @param featureExtractor feature extractor used to extract features and build {@link SimpleDatasetData} + * @param cols number of columns (features) will be extracted + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @param type of a partition {@code context} + * @return dataset */ public static SimpleDataset createSimpleDataset(Map upstreamMap, int partitions, PartitionContextBuilder partCtxBuilder, @@ -178,18 +231,20 @@ public static SimpleDataset createSimpleDatase } /** + * Creates a new instance of local {@link SimpleLabeledDataset} using the specified {@code partCtxBuilder}, + * {@code featureExtractor} and {@code lbExtractor}. This method determines partition {@code data} to be + * {@link SimpleLabeledDatasetData}, but allows to use any desired type of partition {@code context}. * - * - * @param upstreamMap - * @param partitions - * @param partCtxBuilder - * @param featureExtractor - * @param lbExtractor - * @param cols - * @param - * @param - * @param - * @return + * @param upstreamMap {@code Map} with {@code upstream} data + * @param partitions number of partitions {@code upstream} {@code Map} will be divided on + * @param partCtxBuilder partition {@code context} builder + * @param featureExtractor feature extractor used to extract features and build {@link SimpleLabeledDatasetData} + * @param lbExtractor label extractor used to extract labels and buikd {@link SimpleLabeledDatasetData} + * @param cols number of columns (features) will be extracted + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @param type of a partition {@code context} + * @return dataset */ public static SimpleLabeledDataset createSimpleLabeledDataset( Map upstreamMap, int partitions, PartitionContextBuilder partCtxBuilder, @@ -203,15 +258,17 @@ public static SimpleLabeledDataset createSimpl } /** + * Creates a new instance of local {@link SimpleDataset} using the specified {@code featureExtractor}. This + * methods determines partition {@code context} to be {@link EmptyContext} and partition {@code data} to be + * {@link SimpleDatasetData}. * - * - * @param upstreamMap - * @param partitions - * @param featureExtractor - * @param cols - * @param - * @param - * @return + * @param upstreamMap {@code Map} with {@code upstream} data + * @param partitions number of partitions {@code upstream} {@code Map} will be divided on + * @param featureExtractor feature extractor used to extract features and build {@link SimpleDatasetData} + * @param cols number of columns (features) will be extracted + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @return dataset */ public static SimpleDataset createSimpleDataset(Map upstreamMap, int partitions, IgniteBiFunction featureExtractor, int cols) { @@ -219,20 +276,23 @@ public static SimpleDataset createSimpleDataset(Map u } /** + * Creates a new instance of local {@link SimpleLabeledDataset} using the specified {@code featureExtractor} + * and {@code lbExtractor}. This methods determines partition {@code context} to be {@link EmptyContext} and + * partition {@code data} to be {@link SimpleLabeledDatasetData}. * - * - * @param upstreamMap - * @param partitions - * @param featureExtractor - * @param lbExtractor - * @param cols - * @param - * @param - * @return + * @param upstreamMap {@code Map} with {@code upstream} data + * @param partitions number of partitions {@code upstream} {@code Map} will be divided on + * @param featureExtractor feature extractor used to extract features and build {@link SimpleLabeledDatasetData} + * @param lbExtractor label extractor used to extract labels and buikd {@link SimpleLabeledDatasetData} + * @param cols number of columns (features) will be extracted + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @return dataset */ public static SimpleLabeledDataset createSimpleLabeledDataset(Map upstreamMap, int partitions, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor, int cols) { - return createSimpleLabeledDataset(upstreamMap, partitions, new EmptyContextBuilder<>(), featureExtractor, lbExtractor, cols); + return createSimpleLabeledDataset(upstreamMap, partitions, new EmptyContextBuilder<>(), featureExtractor, + lbExtractor, cols); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionContextBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionContextBuilder.java index 10e735ce988bf..a64b5d8585430 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionContextBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionContextBuilder.java @@ -19,24 +19,39 @@ import java.io.Serializable; import java.util.Iterator; +import org.apache.ignite.ml.dataset.api.builder.context.EmptyContextBuilder; import org.apache.ignite.ml.math.functions.IgniteFunction; /** + * Builder that accepts a partition {@code upstream} data and makes partition {@code context}. This builder is used to + * build a partition {@code context} and assumed to be called only once for every partition during a dataset + * initialization. * - * @param - * @param - * @param + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @param type of a partition {@code context} + * + * @see EmptyContextBuilder */ @FunctionalInterface -public interface PartitionContextBuilder { +public interface PartitionContextBuilder extends Serializable { /** + * Builds a new partition {@code context} from an {@code upstream} data. * - * @param upstreamData - * @param upstreamDataSize - * @return + * @param upstreamData partition {@code upstream} data + * @param upstreamDataSize partition {@code upstream} data size + * @return partition {@code context} */ - public C build(Iterator> upstreamData, long upstreamDataSize); + public C build(Iterator> upstreamData, long upstreamDataSize); + /** + * Makes a composed partition {@code context} builder that first builds a {@code context} and then applies the + * specified function on the result. + * + * @param fun function that applied after first partition {@code context} is built + * @param new type of a partition {@code context} + * @return composed partition {@code context} builder + */ default public PartitionContextBuilder andThen(IgniteFunction fun) { return (upstreamData, upstreamDataSize) -> fun.apply(build(upstreamData, upstreamDataSize)); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java index 6f65f9ce8ef54..9cf1d610a8b3b 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java @@ -19,31 +19,43 @@ import java.io.Serializable; import java.util.Iterator; +import org.apache.ignite.ml.dataset.api.builder.data.SimpleDatasetDataBuilder; +import org.apache.ignite.ml.dataset.api.builder.data.SimpleLabeledDatasetDataBuilder; import org.apache.ignite.ml.math.functions.IgniteBiFunction; /** + * Builder that accepts a partition {@code upstream} data and partition {@code context} and makes partition + * {@code data}. This builder is used to build a partition {@code data} and assumed to be called in all cases when + * partition {@code data} not found on the node that performs computation (it might be the result of a previous node + * failure or rebalancing). * - * @param - * @param - * @param - * @param + * @param type of a key in upstream data + * @param type of a value in upstream data + * @param type of a partition context + * @param type of a partition data + * + * @see SimpleDatasetDataBuilder + * @see SimpleLabeledDatasetDataBuilder */ @FunctionalInterface public interface PartitionDataBuilder extends Serializable { /** + * Builds a new partition {@code data} from a partition {@code upstream} data and partition {@code context} * - * @param upstreamData - * @param upstreamDataSize - * @param ctx - * @return + * @param upstreamData partition {@code upstream} data + * @param upstreamDataSize partition {@code upstream} data size + * @param ctx partition {@code context} + * @return partition {@code data} */ - public D build(Iterator> upstreamData, long upstreamDataSize, C ctx); + public D build(Iterator> upstreamData, long upstreamDataSize, C ctx); /** + * Makes a composed partition {@code data} builder that first builds a {@code data} and then applies the specified + * function on the result. * - * @param fun - * @param - * @return + * @param fun function that applied after first partition {@code data} is built + * @param new type of a partition {@code data} + * @return composed partition {@code data} builder */ default public PartitionDataBuilder andThen(IgniteBiFunction fun) { return (upstreamData, upstreamDataSize, ctx) -> fun.apply(build(upstreamData, upstreamDataSize, ctx), ctx); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionUpstreamEntry.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/UpstreamEntry.java similarity index 84% rename from modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionUpstreamEntry.java rename to modules/ml/src/main/java/org/apache/ignite/ml/dataset/UpstreamEntry.java index 4c7288da30682..857a47e25424d 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionUpstreamEntry.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/UpstreamEntry.java @@ -18,12 +18,12 @@ package org.apache.ignite.ml.dataset; /** - * Distributed Learning Context upstream entry. + * Entry of the {@code upstream}. * - * @param type of an upstream value key - * @param type of an upstream value + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data */ -public class PartitionUpstreamEntry { +public class UpstreamEntry { /** Key. */ private final K key; @@ -36,7 +36,7 @@ public class PartitionUpstreamEntry { * @param key key * @param val value */ - public PartitionUpstreamEntry(K key, V val) { + public UpstreamEntry(K key, V val) { this.key = key; this.val = val; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/DatasetWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/DatasetWrapper.java index 00e7a853e4f27..b6270dce248f8 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/DatasetWrapper.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/DatasetWrapper.java @@ -23,23 +23,40 @@ import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; import org.apache.ignite.ml.math.functions.IgniteTriFunction; +/** + * A dataset wrapper that allows to introduce new functionality based on common {@code compute} methods. + * + * @param type of a partition {@code context} + * @param type of a partition {@code data} + * + * @see SimpleDataset + * @see SimpleLabeledDataset + */ public class DatasetWrapper implements Dataset { - + /** Delegate that performs {@code compute} actions. */ protected final Dataset delegate; + /** + * Constructs a new instance of dataset wrapper that delegates {@code compute} actions to the actual delegate. + * + * @param delegate delegate that performs {@code compute} actions + */ public DatasetWrapper(Dataset delegate) { this.delegate = delegate; } + /** {@inheritDoc} */ @Override public R computeWithCtx(IgniteTriFunction map, IgniteBinaryOperator reduce, R identity) { return delegate.computeWithCtx(map, reduce, identity); } + /** {@inheritDoc} */ @Override public R compute(IgniteBiFunction map, IgniteBinaryOperator reduce, R identity) { return delegate.compute(map, reduce, identity); } + /** {@inheritDoc} */ @Override public void close() throws Exception { delegate.close(); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java index 675ea8b4d0d22..ef43bd55e6769 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java @@ -22,13 +22,19 @@ import org.apache.ignite.ml.dataset.Dataset; import org.apache.ignite.ml.dataset.api.data.SimpleDatasetData; +/** + * A simple dataset introduces additional methods based on a matrix of features. + * + * @param type of a partition {@code context} + */ public class SimpleDataset extends DatasetWrapper { /** BLAS (Basic Linear Algebra Subprograms) instance. */ private static final BLAS blas = BLAS.getInstance(); /** + * Creates a new instance of simple dataset that introduces additional methods based on a matrix of features. * - * @param delegate + * @param delegate delegate that performs {@code compute} actions */ public SimpleDataset(Dataset delegate) { super(delegate); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleLabeledDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleLabeledDataset.java index 09bb0b5eb2e15..2fede65054cb6 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleLabeledDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleLabeledDataset.java @@ -21,8 +21,18 @@ import org.apache.ignite.ml.dataset.Dataset; import org.apache.ignite.ml.dataset.api.data.SimpleLabeledDatasetData; +/** + * A simple labeled dataset introduces additional methods based on a matrix of features and labels vector. + * + * @param type of a partition {@code context} + */ public class SimpleLabeledDataset extends DatasetWrapper { - + /** + * Creates a new instance of simple labeled dataset that introduces additional methods based on a matrix of features + * and labels vector. + * + * @param delegate delegate that performs {@code compute} actions + */ public SimpleLabeledDataset(Dataset delegate) { super(delegate); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/EmptyContextBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/EmptyContextBuilder.java index ddef16c3589de..894013dd534bd 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/EmptyContextBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/EmptyContextBuilder.java @@ -19,12 +19,21 @@ import java.util.Iterator; import org.apache.ignite.ml.dataset.PartitionContextBuilder; -import org.apache.ignite.ml.dataset.PartitionUpstreamEntry; +import org.apache.ignite.ml.dataset.UpstreamEntry; import org.apache.ignite.ml.dataset.api.context.EmptyContext; +/** + * A partition {@code context} builder that makes {@link EmptyContext}. + * + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + */ public class EmptyContextBuilder implements PartitionContextBuilder { + /** */ + private static final long serialVersionUID = 6620781747993467186L; - @Override public EmptyContext build(Iterator> upstreamData, long upstreamDataSize) { + /** {@inheritDoc} */ + @Override public EmptyContext build(Iterator> upstreamData, long upstreamDataSize) { return new EmptyContext(); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleDatasetDataBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleDatasetDataBuilder.java index 503a022a97873..ab154a4fcfebc 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleDatasetDataBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleDatasetDataBuilder.java @@ -20,36 +20,53 @@ import java.io.Serializable; import java.util.Iterator; import org.apache.ignite.ml.dataset.PartitionDataBuilder; -import org.apache.ignite.ml.dataset.PartitionUpstreamEntry; +import org.apache.ignite.ml.dataset.UpstreamEntry; import org.apache.ignite.ml.dataset.api.data.SimpleDatasetData; import org.apache.ignite.ml.math.functions.IgniteBiFunction; +/** + * A partition {@code data} builder that makes {@link SimpleDatasetData}. + * + * @param type of a key in upstream data + * @param type of a value in upstream data + * @param type of a partition context + */ public class SimpleDatasetDataBuilder implements PartitionDataBuilder { + /** */ + private static final long serialVersionUID = 756800193212149975L; + /** Function that extracts features from an {@code upstream} data. */ private final IgniteBiFunction featureExtractor; + /** Number of columns (features). */ private final int cols; + /** + * Construct a new instance of partition {@code data} builder that makes {@link SimpleDatasetData}. + * + * @param featureExtractor function that extracts features from an {@code upstream} data + * @param cols number of columns (features) + */ public SimpleDatasetDataBuilder(IgniteBiFunction featureExtractor, int cols) { this.featureExtractor = featureExtractor; this.cols = cols; } - /** */ - @Override public SimpleDatasetData build(Iterator> upstreamData, long upstreamDataSize, - C ctx) { + /** {@inheritDoc} */ + @Override public SimpleDatasetData build(Iterator> upstreamData, long upstreamDataSize, C ctx) { + // Prepares the matrix of features in flat column-major format. double[] features = new double[Math.toIntExact(upstreamDataSize * cols)]; int ptr = 0; while (upstreamData.hasNext()) { - PartitionUpstreamEntry entry = upstreamData.next(); + UpstreamEntry entry = upstreamData.next(); double[] row = featureExtractor.apply(entry.getKey(), entry.getValue()); - assert row.length == cols; + assert row.length == cols : "Feature extractor must return exactly " + cols + " features"; for (int i = 0; i < cols; i++) - features[Math.toIntExact(i * upstreamDataSize) + ptr] = row[i]; + features[Math.toIntExact(i * upstreamDataSize + ptr)] = row[i]; ptr++; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleLabeledDatasetDataBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleLabeledDatasetDataBuilder.java index 6becec77ab3d2..fa63f453336bf 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleLabeledDatasetDataBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleLabeledDatasetDataBuilder.java @@ -20,19 +20,38 @@ import java.io.Serializable; import java.util.Iterator; import org.apache.ignite.ml.dataset.PartitionDataBuilder; -import org.apache.ignite.ml.dataset.PartitionUpstreamEntry; +import org.apache.ignite.ml.dataset.UpstreamEntry; import org.apache.ignite.ml.dataset.api.data.SimpleLabeledDatasetData; import org.apache.ignite.ml.math.functions.IgniteBiFunction; +/** + * A partition {@code data} builder that makes {@link SimpleLabeledDatasetData}. + * + * @param type of a key in upstream data + * @param type of a value in upstream data + * @param type of a partition context + */ public class SimpleLabeledDatasetDataBuilder implements PartitionDataBuilder { + /** */ + private static final long serialVersionUID = 3678784980215216039L; + /** Function that extracts features from an {@code upstream} data. */ private final IgniteBiFunction featureExtractor; + /** Function that extracts labels from an {@code upstream} data. */ private final IgniteBiFunction lbExtractor; + /** Number of columns (features). */ private final int cols; + /** + * Constructs a new instance of partition {@code data} builder that makes {@link SimpleLabeledDatasetData}. + * + * @param featureExtractor function that extracts features from an {@code upstream} data + * @param lbExtractor function that extracts labels from an {@code upstream} data + * @param cols number of columns (features) + */ public SimpleLabeledDatasetDataBuilder(IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor, int cols) { this.featureExtractor = featureExtractor; @@ -40,18 +59,19 @@ public SimpleLabeledDatasetDataBuilder(IgniteBiFunction featureE this.cols = cols; } - /** */ - @Override public SimpleLabeledDatasetData build(Iterator> upstreamData, + /** {@inheritDoc} */ + @Override public SimpleLabeledDatasetData build(Iterator> upstreamData, long upstreamDataSize, C ctx) { + // Prepares the matrix of features in flat column-major format. double[] features = new double[Math.toIntExact(upstreamDataSize * cols)]; double[] labels = new double[Math.toIntExact(upstreamDataSize)]; int ptr = 0; while (upstreamData.hasNext()) { - PartitionUpstreamEntry entry = upstreamData.next(); + UpstreamEntry entry = upstreamData.next(); double[] row = featureExtractor.apply(entry.getKey(), entry.getValue()); - assert row.length == cols; + assert row.length == cols : "Feature extractor must return exactly " + cols + " features"; for (int i = 0; i < cols; i++) features[Math.toIntExact(i * upstreamDataSize) + ptr] = row[i]; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/EmptyContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/EmptyContext.java index 5cb436dcb2ebe..795be142d7e60 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/EmptyContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/EmptyContext.java @@ -19,5 +19,10 @@ import java.io.Serializable; +/** + * An empty partition {@code context}. + */ public class EmptyContext implements Serializable { + /** */ + private static final long serialVersionUID = 4108938672110578991L; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleDatasetData.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleDatasetData.java index 7d5fb80433487..d174c50588b43 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleDatasetData.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleDatasetData.java @@ -17,6 +17,12 @@ package org.apache.ignite.ml.dataset.api.data; +import org.apache.ignite.ml.dataset.api.SimpleDataset; + +/** + * A partition {@code data} of the {@link SimpleDataset} containing matrix of features in flat column-major format + * stored in heap. + */ public class SimpleDatasetData implements AutoCloseable { /** Matrix of features in a dense flat column-major format. */ private final double[] features; @@ -28,7 +34,8 @@ public class SimpleDatasetData implements AutoCloseable { private final int cols; /** - * Constructs a new instance of recoverable data of DLC labeled dataset partition. + * Constructs a new instance of partition {@code data} of the {@link SimpleDataset} containing matrix of features in + * flat column-major format stored in heap. * * @param features matrix of features in a dense flat column-major format * @param rows number of rows @@ -57,6 +64,6 @@ public int getCols() { /** {@inheritDoc} */ @Override public void close() { - // do nothing + // Do nothing, GC will clean up. } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleLabeledDatasetData.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleLabeledDatasetData.java index 2dbbfa1fa1962..eb40a30d3d7ce 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleLabeledDatasetData.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleLabeledDatasetData.java @@ -17,6 +17,12 @@ package org.apache.ignite.ml.dataset.api.data; +import org.apache.ignite.ml.dataset.api.SimpleLabeledDataset; + +/** + * A partition {@code data} of the {@link SimpleLabeledDataset} containing matrix of features in flat column-major + * format stored in heap and vector of labels stored in heap as well. + */ public class SimpleLabeledDatasetData implements AutoCloseable { /** Matrix with features in a dense flat column-major format. */ private final double[] features; @@ -31,11 +37,13 @@ public class SimpleLabeledDatasetData implements AutoCloseable { private final double[] labels; /** + * Constructs a new instance of partition {@code data} of the {@link SimpleLabeledDataset} containing matrix of + * features in flat column-major format stored in heap and vector of labels stored in heap as well. * - * @param features - * @param rows - * @param cols - * @param labels + * @param features matrix with features in a dense flat column-major format + * @param rows number of rows + * @param cols number of columns + * @param labels vector with labels */ public SimpleLabeledDatasetData(double[] features, int rows, int cols, double[] labels) { this.features = features; @@ -59,12 +67,13 @@ public int getCols() { return cols; } + /** */ public double[] getLabels() { return labels; } /** {@inheritDoc} */ @Override public void close() { - // do nothing + // Do nothing, GC will clean up. } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java index de4eb5f3e3565..cb052e9ec811e 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java @@ -33,42 +33,46 @@ import org.apache.ignite.ml.math.functions.IgniteTriFunction; /** + * An implementation of dataset based on Ignite Cache, which is used as {@code upstream} and as reliable storage for + * partition {@code context} as well. * - * @param - * @param - * @param - * @param + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @param type of a partition {@code context} + * @param type of a partition {@code data} */ public class CacheBasedDataset implements Dataset { - /** */ + /** Number of retries for the case when one of partitions not found on the node where computation is performed. */ private static final int RETRIES = 100; - /** */ + /** Retry interval (ms) for the case when one of partitions not found on the node where computation is performed. */ private static final int RETRY_INTERVAL = 500; - /** */ + /** Ignite instance. */ private final Ignite ignite; - /** */ + /** Ignite Cache with {@code upstream} data. */ private final IgniteCache upstreamCache; - /** */ + /** Ignite Cache with partition {@code context}. */ private final IgniteCache datasetCache; - /** */ + /** Partition {@code data} builder. */ private final PartitionDataBuilder partDataBuilder; - /** */ + /** Dataset ID that is used to identify dataset in local storage on the node where computation is performed. */ private final UUID datasetId; /** + * Constructs a new instance of dataset based on Ignite Cache, which is used as {@code upstream} and as reliable storage for + * partition {@code context} as well. * - * @param ignite - * @param upstreamCache - * @param datasetCache - * @param partDataBuilder - * @param datasetId + * @param ignite Ignite instance + * @param upstreamCache Ignite Cache with {@code upstream} data + * @param datasetCache Ignite Cache with partition {@code context} + * @param partDataBuilder partition {@code data} builder + * @param datasetId dataset ID */ public CacheBasedDataset(Ignite ignite, IgniteCache upstreamCache, IgniteCache datasetCache, PartitionDataBuilder partDataBuilder, @@ -126,12 +130,15 @@ public CacheBasedDataset(Ignite ignite, IgniteCache upstreamCache, } /** + * Calls the {@code MapReduce} job specified as the {@code fun} function and the {@code reduce} reducer on all + * partitions with guarantee that partitions with the same index of upstream and partition {@code context} caches + * will be on the same node during the computation and will not be moved before computation is finished. * - * @param fun - * @param reduce - * @param identity - * @param - * @return + * @param fun function that applies to all partitions + * @param reduce function that reduces results of {@code fun} + * @param identity identity + * @param type of a result + * @return final result */ private R computeForAllPartitions(IgniteFunction fun, IgniteBinaryOperator reduce, R identity) { Collection cacheNames = Arrays.asList(datasetCache.getName(), upstreamCache.getName()); @@ -143,4 +150,14 @@ private R computeForAllPartitions(IgniteFunction fun, IgniteBina return res; } + + /** */ + public IgniteCache getUpstreamCache() { + return upstreamCache; + } + + /** */ + public IgniteCache getDatasetCache() { + return datasetCache; + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java index 69b8f60baf565..2c64d542343bb 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java @@ -23,7 +23,6 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.affinity.AffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.ml.dataset.Dataset; import org.apache.ignite.ml.dataset.DatasetBuilder; import org.apache.ignite.ml.dataset.PartitionContextBuilder; import org.apache.ignite.ml.dataset.PartitionDataBuilder; @@ -31,30 +30,45 @@ import org.apache.ignite.ml.dataset.impl.cache.util.DatasetAffinityFunctionWrapper; /** + * A dataset builder that makes {@link CacheBasedDataset}. Encapsulate logic of building cache based dataset such as + * allocation required data structures and initialization of {@code context} part of partitions. * - * @param - * @param - * @param - * @param + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @param type of a partition {@code context} + * @param type of a partition {@code data} */ public class CacheBasedDatasetBuilder implements DatasetBuilder { - + /** Number of retries for the case when one of partitions not found on the node where loading is performed. */ private static final int RETRIES = 100; + /** Retry interval (ms) for the case when one of partitions not found on the node where loading is performed. */ private static final int RETRY_INTERVAL = 500; + /** Template of the name of Ignite Cache containing partition {@code context}. */ private static final String DATASET_CACHE_TEMPLATE = "%s_DATASET_%s"; /** Ignite instance. */ private final Ignite ignite; + /** Ignite Cache with {@code upstream} data. */ private final IgniteCache upstreamCache; + /** Partition {@code context} builder. */ private final PartitionContextBuilder partCtxBuilder; + /** Partition {@code data} builder. */ private final PartitionDataBuilder partDataBuilder; + /** + * Constructs a new instance of cache based dataset builder that makes {@link CacheBasedDataset}. + * + * @param ignite Ignite instance + * @param upstreamCache Ignite Cache with {@code upstream} data + * @param partCtxBuilder Ignite Cache with partition {@code context} + * @param partDataBuilder Partition {@code data} builder + */ public CacheBasedDatasetBuilder(Ignite ignite, IgniteCache upstreamCache, PartitionContextBuilder partCtxBuilder, PartitionDataBuilder partDataBuilder) { this.ignite = ignite; @@ -63,13 +77,17 @@ public CacheBasedDatasetBuilder(Ignite ignite, IgniteCache upstreamCache, this.partDataBuilder = partDataBuilder; } + /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public Dataset build() { + @Override public CacheBasedDataset build() { UUID datasetId = UUID.randomUUID(); + // Retrieves affinity function of the upstream Ignite Cache. CacheConfiguration upstreamCacheConfiguration = upstreamCache.getConfiguration(CacheConfiguration.class); AffinityFunction upstreamCacheAffinity = upstreamCacheConfiguration.getAffinity(); + // Creates dataset cache configuration with affinity function that mimics to affinity function of the upstream + // cache. CacheConfiguration datasetCacheConfiguration = new CacheConfiguration<>(); datasetCacheConfiguration.setName(String.format(DATASET_CACHE_TEMPLATE, upstreamCache.getName(), datasetId)); datasetCacheConfiguration.setAffinity(new DatasetAffinityFunctionWrapper(upstreamCacheAffinity)); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java index 17bcaf217a5af..e092b77bbff61 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java @@ -41,20 +41,25 @@ import org.apache.ignite.ml.math.functions.IgniteFunction; /** - * + * Util class that provides common methods to perform computations on top of the Ignite Compute Grid. */ public class ComputeUtils { - - private static final String DATA_STORAGE_KEY = "part_data_storage_%s"; + /** Template of the key used to store partition {@code data} in local storage. */ + private static final String DATA_STORAGE_KEY_TEMPLATE = "part_data_storage_%s"; /** - * @param ignite - * @param cacheNames - * @param fun - * @param retries - * @param interval - * @param - * @return + * Calls the specified {@code fun} function on all partitions so that is't guaranteed that partitions with the same + * index of all specified caches will be placed on the same node and will not be moved before computation is + * finished. If partitions are placed on different nodes then call will be retried, but not more than {@code + * retries} times with {@code interval} interval specified in milliseconds. + * + * @param ignite Ignite instance + * @param cacheNames collection of cache names + * @param fun function to be applied on all partitions + * @param retries number of retries for the case when one of partitions not found on the node + * @param interval interval of retries for the case when one of partitions not found on the node + * @param type of a result + * @return collection of results */ public static Collection affinityCallWithRetries(Ignite ignite, Collection cacheNames, IgniteFunction fun, int retries, int interval) { @@ -86,7 +91,7 @@ public static Collection affinityCallWithRetries(Ignite ignite, Collectio } // Collects results. - for (int part : futures.keySet()) { + for (int part : futures.keySet()) try { R res = futures.get(part).get(); results.add(res); @@ -94,7 +99,6 @@ public static Collection affinityCallWithRetries(Ignite ignite, Collectio } catch (PartitionNotFoundException ignore) { } - } if (completionFlags.cardinality() == partitions) return results; @@ -106,12 +110,17 @@ public static Collection affinityCallWithRetries(Ignite ignite, Collectio } /** - * @param ignite - * @param cacheNames - * @param fun - * @param retries - * @param - * @return + * Calls the specified {@code fun} function on all partitions so that is't guaranteed that partitions with the same + * index of all specified caches will be placed on the same node and will not be moved before computation is + * finished. If partitions are placed on different nodes then call will be retried, but not more than {@code + * retries} times. + * + * @param ignite Ignite instance + * @param cacheNames collection of cache names + * @param fun function to be applied on all partitions + * @param retries number of retries for the case when one of partitions not found on the node + * @param type of a result + * @return collection of results */ public static Collection affinityCallWithRetries(Ignite ignite, Collection cacheNames, IgniteFunction fun, int retries) { @@ -119,27 +128,30 @@ public static Collection affinityCallWithRetries(Ignite ignite, Collectio } /** - * @param ignite - * @param upstreamCacheName - * @param datasetCacheName - * @param learningCtxId - * @param part - * @param partDataBuilder - * @param - * @param - * @param - * @param - * @return + * Extracts partition {@code data} from the local storage, if it's not found in local storage recovers this {@code + * data} from a partition {@code upstream} and {@code context}. + * + * @param ignite Ignite instance + * @param upstreamCacheName name of an {@code upstream} cache + * @param datasetCacheName name of a partition {@code context} cache + * @param datasetId dataset ID + * @param part partition index + * @param partDataBuilder partition data builder + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @param type of a partition {@code context} + * @param type of a partition {@code data} + * @return partition {@code data} */ @SuppressWarnings("unchecked") public static D getData(Ignite ignite, - String upstreamCacheName, String datasetCacheName, UUID learningCtxId, int part, + String upstreamCacheName, String datasetCacheName, UUID datasetId, int part, PartitionDataBuilder partDataBuilder) { PartitionDataStorage dataStorage = (PartitionDataStorage)ignite .cluster() .nodeLocalMap() - .computeIfAbsent(String.format(DATA_STORAGE_KEY, learningCtxId), key -> new PartitionDataStorage()); + .computeIfAbsent(String.format(DATA_STORAGE_KEY_TEMPLATE, datasetId), key -> new PartitionDataStorage()); Object data = dataStorage.computeDataIfAbsent(part, () -> { IgniteCache learningCtxCache = ignite.cache(datasetCacheName); @@ -162,14 +174,15 @@ public static D getData( } /** + * Initializes partition {@code context} by loading it from a partition {@code upstream}. * - * @param ignite - * @param upstreamCacheName - * @param datasetCacheName - * @param ctxBuilder - * @param - * @param - * @param + * @param ignite Ignite instance + * @param upstreamCacheName name of an {@code upstream} cache + * @param datasetCacheName name of a partition {@code context} cache + * @param ctxBuilder partition {@code context} builder + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @param type of a partition {@code context} */ public static void initContext(Ignite ignite, String upstreamCacheName, String datasetCacheName, PartitionContextBuilder ctxBuilder, int retries, int interval) { @@ -188,7 +201,7 @@ public static void initContext(Ignite ignite, Str ctx = ctxBuilder.build(new UpstreamCursorAdapter<>(cursor.iterator()), cnt); } - IgniteCache datasetCache = ignite.cache(datasetCacheName); + IgniteCache datasetCache = locIgnite.cache(datasetCacheName); datasetCache.put(part, ctx); @@ -197,15 +210,16 @@ public static void initContext(Ignite ignite, Str } /** + * Initializes partition {@code context} by loading it from a partition {@code upstream}. * - * @param ignite - * @param upstreamCacheName - * @param datasetCacheName - * @param ctxBuilder - * @param retries - * @param - * @param - * @param + * @param ignite Ignite instance + * @param upstreamCacheName name of an {@code upstream} cache + * @param datasetCacheName name of a partition {@code context} cache + * @param ctxBuilder partition {@code context} builder + * @param retries number of retries for the case when one of partitions not found on the node + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @param type of a partition {@code context} */ public static void initContext(Ignite ignite, String upstreamCacheName, String datasetCacheName, PartitionContextBuilder ctxBuilder, int retries) { @@ -213,10 +227,12 @@ public static void initContext(Ignite ignite, Str } /** - * @param ignite - * @param datasetCacheName - * @param part - * @param + * Extracts partition {@code context} from the Ignite Cache. + * + * @param ignite Ignite instance + * @param datasetCacheName collection of cache names + * @param part partition index + * @param type of a partition {@code context} * @return */ public static C getContext(Ignite ignite, String datasetCacheName, int part) { @@ -225,9 +241,14 @@ public static C getContext(Ignite ignite, String datase } /** - * @param ignite - * @param cacheNames - * @param part + * Checks that partitions with the specified partition index of all caches are placed on the same node. In case of + * rebalancing it's not guaranteed that partitions of caches even with the same affinity function will be moved + * synchronously. A workaround used here is based on optimistic locking with checking that partitions available on + * the node. + * + * @param ignite Ignite instance + * @param cacheNames collection of cache names + * @param part partition index */ private static void checkAllPartitionsAvailable(Ignite ignite, Collection cacheNames, int part) { for (String cacheName : cacheNames) { diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/DatasetAffinityFunctionWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/DatasetAffinityFunctionWrapper.java index 0d8c45a04e3d4..32aeb6258f026 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/DatasetAffinityFunctionWrapper.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/DatasetAffinityFunctionWrapper.java @@ -24,20 +24,20 @@ import org.apache.ignite.cluster.ClusterNode; /** - * Affinity function wrapper which uses key as a partition index and delegates all other functions to specified + * Affinity function wrapper that uses key as a partition index and delegates all other functions to specified * delegate. */ public class DatasetAffinityFunctionWrapper implements AffinityFunction { /** */ private static final long serialVersionUID = -8233787063079973753L; - /** Delegate which actually performs all methods except {@link #partition(Object)}. */ + /** Delegate that actually performs all methods except {@code partition()}. */ private final AffinityFunction delegate; /** - * Constructs a new instance of a DLC affinity function wrapper. + * Constructs a new instance of affinity function wrapper. * - * @param delegate affinity function which actually performs all methods except {@link #partition(Object)} + * @param delegate affinity function which actually performs all methods except {@code partition()} */ public DatasetAffinityFunctionWrapper(AffinityFunction delegate) { this.delegate = delegate; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java index 45282cf91c11e..2a636bea749cf 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java @@ -21,25 +21,29 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; -import org.apache.ignite.ml.math.functions.IgniteSupplier; +import java.util.function.Function; +import java.util.function.Supplier; /** - * + * Local storage used to keep partition {@code data}. */ -public class PartitionDataStorage { - /** */ +class PartitionDataStorage { + /** Storage of a partition {@code data}. */ private final ConcurrentMap storage = new ConcurrentHashMap<>(); - /** */ + /** Storage of locks correspondent to partition {@code data} objects. */ private final ConcurrentMap locks = new ConcurrentHashMap<>(); /** + * Retrieves partition {@code data} correspondent to specified partition index if it exists in local storage or + * loads it using the specified {@code supplier}. Unlike {@link ConcurrentMap#computeIfAbsent(Object, Function)}, + * this method guarantees that function will be called only once. * - * @param part - * @param supplier - * @return + * @param part partition index + * @param supplier partition {@code data} supplier. + * @return partition {@code data} */ - public Object computeDataIfAbsent(int part, IgniteSupplier supplier) { + Object computeDataIfAbsent(int part, Supplier supplier) { Object data = storage.get(part); if (data == null) { diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionNotFoundException.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionNotFoundException.java index e5037e8014bc6..96fd2adab3cf6 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionNotFoundException.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionNotFoundException.java @@ -22,15 +22,15 @@ import org.apache.ignite.IgniteException; /** - * Exception which is thrown when partition is expected to be on the node but it isn't. Assumed reason is that two - * caches with the same affinity function have all their partitions on the same nodes only in terms of eventual - * consistency. + * Exception which is thrown when partition is expected to be on the node but it isn't. In case of rebalancing it's not + * guaranteed that partitions of caches even with the same affinity function will be moved synchronously. A workaround + * used here is based on optimistic locking with checking that partitions available on the node. */ -public class PartitionNotFoundException extends IgniteException implements Serializable { +class PartitionNotFoundException extends IgniteException implements Serializable { /** */ private static final long serialVersionUID = -8891869046312827676L; - /** Exception message template. */ + /** templace of an exception message.. */ private static final String MSG_TEMPLATE = "Partition %d of %s expected to be on node %s, but it isn't"; /** @@ -40,7 +40,7 @@ public class PartitionNotFoundException extends IgniteException implements Seria * @param nodeId node id * @param partIdx partition index */ - public PartitionNotFoundException(String cacheName, UUID nodeId, int partIdx) { + PartitionNotFoundException(String cacheName, UUID nodeId, int partIdx) { super(String.format(MSG_TEMPLATE, partIdx, cacheName, nodeId.toString())); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java index dba8375f9563b..19ff7ea3930ab 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java @@ -19,16 +19,16 @@ import java.util.Iterator; import javax.cache.Cache; -import org.apache.ignite.ml.dataset.PartitionUpstreamEntry; +import org.apache.ignite.ml.dataset.UpstreamEntry; /** * Cursor adapter used to transform {@link Cache.Entry} received from Ignite Cache query cursor into DLC-specific - * {@link PartitionUpstreamEntry}. + * {@link UpstreamEntry}. * * @param type of an upstream value key * @param type of an upstream value */ -public class UpstreamCursorAdapter implements Iterator> { +public class UpstreamCursorAdapter implements Iterator> { /** Cache entry iterator. */ private final Iterator> delegate; @@ -47,12 +47,12 @@ public class UpstreamCursorAdapter implements Iterator next() { + @Override public UpstreamEntry next() { Cache.Entry next = delegate.next(); if (next == null) return null; - return new PartitionUpstreamEntry<>(next.getKey(), next.getValue()); + return new UpstreamEntry<>(next.getKey(), next.getValue()); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java index 8137251b7546c..d0baadeb2c13a 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java @@ -24,17 +24,33 @@ import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; import org.apache.ignite.ml.math.functions.IgniteTriFunction; +/** + * An implementation of dataset based on local data structures such as {@code Map} and {@code List} and doesn't requires + * Ignite environment. Introduces for testing purposes mostly, but can be used for simple local computations as well. + * + * @param type of a partition {@code context} + * @param type of a partition {@code data} + */ public class LocalDataset implements Dataset { - + /** Partition {@code context} storage. */ private final List ctx; + /** Partition {@code data} storage. */ private final List data; - public LocalDataset(List ctx, List data) { + /** + * Constructs a new instance of dataset based on local data structures such as {@code Map} and {@code List} and + * doesn't requires Ignite environment. + * + * @param ctx partition {@code context} storage + * @param data partition {@code data} storage + */ + LocalDataset(List ctx, List data) { this.ctx = ctx; this.data = data; } + /** {@inheritDoc} */ @Override public R computeWithCtx(IgniteTriFunction map, IgniteBinaryOperator reduce, R identity) { R res = identity; @@ -44,6 +60,7 @@ public LocalDataset(List ctx, List data) { return res; } + /** {@inheritDoc} */ @Override public R compute(IgniteBiFunction map, IgniteBinaryOperator reduce, R identity) { R res = identity; @@ -53,7 +70,18 @@ public LocalDataset(List ctx, List data) { return res; } + /** {@inheritDoc} */ @Override public void close() { - // Do nothing. + // Do nothing, GC will clean up. + } + + /** */ + public List getCtx() { + return ctx; + } + + /** */ + public List getData() { + return data; } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java index 25ee1430e5b53..387d754dafd1d 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java @@ -22,24 +22,43 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import org.apache.ignite.ml.dataset.Dataset; import org.apache.ignite.ml.dataset.DatasetBuilder; import org.apache.ignite.ml.dataset.PartitionContextBuilder; import org.apache.ignite.ml.dataset.PartitionDataBuilder; -import org.apache.ignite.ml.dataset.PartitionUpstreamEntry; +import org.apache.ignite.ml.dataset.UpstreamEntry; import org.apache.ignite.ml.math.functions.IgniteFunction; +/** + * A dataset builder that makes {@link LocalDataset}. Encapsulate logic of building local dataset such as allocation + * required data structures and initialization of {@code context} part of partitions. + * + * @param type of a key in {@code upstream} data + * @param type of a value in {@code upstream} data + * @param type of a partition {@code context} + * @param type of a partition {@code data} + */ public class LocalDatasetBuilder implements DatasetBuilder { - + /** {@code Map} with upstream data. */ private final Map upstreamMap; + /** Number of partitions. */ private final int partitions; + /** Partition {@code context} builder. */ private final PartitionContextBuilder partCtxBuilder; + /** Partition {@code data} builder. */ private final PartitionDataBuilder partDataBuilder; + /** + * Constructs a new instance of local dataset builder that makes {@link LocalDataset}. + * + * @param upstreamMap {@code Map} with upstream data + * @param partitions number of partitions + * @param partCtxBuilder partition {@code context} builder + * @param partDataBuilder partition {@code data} builder + */ public LocalDatasetBuilder(Map upstreamMap, int partitions, PartitionContextBuilder partCtxBuilder, PartitionDataBuilder partDataBuilder) { this.upstreamMap = upstreamMap; @@ -50,57 +69,78 @@ public LocalDatasetBuilder(Map upstreamMap, int partitions, /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public Dataset build() { - List ctx = new ArrayList<>(); - List data = new ArrayList<>(); + @Override public LocalDataset build() { + List ctxList = new ArrayList<>(); + List dataList = new ArrayList<>(); int partSize = upstreamMap.size() / partitions; Iterator firstKeysIter = upstreamMap.keySet().iterator(); Iterator secondKeysIter = upstreamMap.keySet().iterator(); + for (int part = 0; part < partitions; part++) { int cnt = Math.min((part + 1) * partSize, upstreamMap.size()) - part * partSize; - C c = partCtxBuilder.build( - new IteratorWindow<>(firstKeysIter, k -> new PartitionUpstreamEntry<>(k, upstreamMap.get(k)), cnt), + C ctx = partCtxBuilder.build( + new IteratorWindow<>(firstKeysIter, k -> new UpstreamEntry<>(k, upstreamMap.get(k)), cnt), cnt ); - D d = partDataBuilder.build( - new IteratorWindow<>(secondKeysIter, k -> new PartitionUpstreamEntry<>(k, upstreamMap.get(k)), cnt), + D data = partDataBuilder.build( + new IteratorWindow<>(secondKeysIter, k -> new UpstreamEntry<>(k, upstreamMap.get(k)), cnt), cnt, - c + ctx ); - ctx.add(c); - data.add(d); + ctxList.add(ctx); + dataList.add(data); } - return new LocalDataset<>(ctx, data); + return new LocalDataset<>(ctxList, dataList); } + /** + * Utils class that wraps iterator so that it produces only specified number of entries and allows to transform + * entries from one type to another. + * + * @param initial type of entries + * @param target type of entries + */ private static class IteratorWindow implements Iterator { - + /** Delegate iterator. */ private final Iterator delegate; + /** Transformer that transforms entries from one type to another. */ private final IgniteFunction map; + /** Count of entries to produce. */ private final int cnt; + /** Number of already produced entries. */ private int ptr; - public IteratorWindow(Iterator delegate, IgniteFunction map, int cnt) { + /** + * Constructs a new instance of iterator window wrapper. + * + * @param delegate delegate iterator + * @param map transformer that transforms entries from one type to another + * @param cnt count of entries to produce + */ + IteratorWindow(Iterator delegate, IgniteFunction map, int cnt) { this.delegate = delegate; this.map = map; this.cnt = cnt; } + /** {@inheritDoc} */ @Override public boolean hasNext() { return delegate.hasNext() && ptr < cnt; } + /** {@inheritDoc} */ @Override public T next() { - ptr++; + ++ptr; + return map.apply(delegate.next()); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java deleted file mode 100644 index 2bb21329ad251..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContext.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * 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.ml.dlearn; - -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; -import org.apache.ignite.ml.math.functions.IgniteBiConsumer; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; -import org.apache.ignite.ml.math.functions.IgniteConsumer; -import org.apache.ignite.ml.math.functions.IgniteFunction; - -/** - * Learning context is a context maintained during a whole learning or data analysis process. The context provides an - * ability to perform calculations in the map-reduce manner and guarantees maintenance of the partition states between - * {@code compute()} calls. - * - * @param

type of a learning context partition - */ -public interface DLearnContext

extends AutoCloseable { - /** - * Computes a given function on every d-learn partition in current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. - * - * @param mapper mapper function applied on every partition - * @param reducer reducer of the results - * @param identity identity value - * @param result type - * @return final reduced result - */ - public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer, R identity); - - /** - * Computes a given function on every d-learn partition in current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. - * - * @param mapper mapper function applied on every partition - * @param reducer reducer of the results - * @param result type - * @return final reduced result - */ - default public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer) { - return compute(mapper, reducer, null); - } - - /** - * Computes a given function on every d-learn partition in current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. - * - * @param mapper mapper function applied on every partition - * @param reducer reducer of the results - * @param identity identity value - * @param result type - * @return final reduced result - */ - default public R compute(IgniteFunction mapper, IgniteBinaryOperator reducer, R identity) { - return compute((part, partIdx) -> mapper.apply(part), reducer, identity); - } - - /** - * Computes a given function on every d-learn partition in current learning context independently and then reduces - * results into one final single result. The goal of this approach is to perform {@code mapper} locally on the nodes - * where partitions are placed and do not involve network subsystem where it's possible. - * - * @param mapper mapper function applied on every partition - * @param reducer reducer of the results - * @param result type - * @return final reduced result - */ - default public R compute(IgniteFunction mapper, IgniteBinaryOperator reducer) { - return compute((part, partIdx) -> mapper.apply(part), reducer); - } - - /** - * Computes a given function on every d-learn partition in current learning context independently. The goal of this - * approach is to perform {@code mapper} locally on the nodes where partitions are placed and do not involve network - * subsystem where it's possible. - * - * @param mapper mapper function applied on every partition - */ - public void compute(IgniteBiConsumer mapper); - - /** - * Computes a given function on every d-learn partition in current learning context independently. The goal of this - * approach is to perform {@code mapper} locally on the nodes where partitions are placed and do not involve network - * subsystem where it's possible. - * - * @param mapper mapper function applied on every partition - */ - default public void compute(IgniteConsumer

mapper) { - compute((part, partIdx) -> mapper.accept(part)); - } - - /** - * Transforms current learning context into another learning context which contains another type of d-learn - * partitions. Transformation doesn't involve new cache instantiation or network data transfer, it just performs - * {@code #transform(IgniteBiConsumer, DLearnPartitionFactory)} locally on every partition in the current context - * and saves results into the same context cache, but with a new context id. - * - * @param transformer transformer - * @param type of new d-learn partition - * @return new learning context - */ - public > C transform( - DLearnContextTransformer transformer); - - /** - * Removes all data associated with the context. - */ - public void close(); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContextFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContextFactory.java deleted file mode 100644 index bc0c1503b3129..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnContextFactory.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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.ml.dlearn; - -import java.io.Serializable; - -/** - * Factory produces learning context instances (see {@link DLearnContext}). - * - * @param

type of a d-learn partition - */ -@FunctionalInterface -public interface DLearnContextFactory

extends Serializable { - /** - * Constructs a new instance of learning context. - * - * @return learning context - */ - public DLearnContext

createContext(); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionFactory.java deleted file mode 100644 index b30f60ba1e8da..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionFactory.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.ml.dlearn; - -import java.io.Serializable; - -/** - * With assumption that a d-lear partition is a type-safe wrapper on top of a partition storage (distributed or not), - * {@code DLearnPartitionFactory} provides API for instantiation of a new d-learn partition instances based on the given - * partition storage. - * - * @param

type of a d-learn partition - */ -@FunctionalInterface -public interface DLearnPartitionFactory

extends Serializable { - /** - * Creates a new d-learn partition (type-safe wrapper on top of a partition storage). - * - * @param storage partition storage - * @return d-learn partition - */ - public P createPartition(DLearnPartitionStorage storage); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionStorage.java deleted file mode 100644 index a4bf70f0b57e0..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/DLearnPartitionStorage.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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.ml.dlearn; - -/** - * D-learn partition storage is a common interface for partition storages both local and distributed. Partition storage - * allows to save, retrieve and remove objects identified by keys from the d-learn partition. - */ -public interface DLearnPartitionStorage { - /** - * Saves the given value in the d-learn partition with the given key. - * - * @param key key - * @param val value - * @param type of value - */ - public void put(String key, T val); - - /** - * Retrieves value from the d-learn partition by the given key. - * - * @param key key - * @param type of value - * @return value - */ - public T get(String key); - - /** - * Removes value from the d-learn partition by the given key. - * - * @param key key - */ - public void remove(String key); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java deleted file mode 100644 index db86c890fce32..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContext.java +++ /dev/null @@ -1,202 +0,0 @@ -/* - * 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.ml.dlearn.context.cache; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.UUID; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.cluster.ClusterGroup; -import org.apache.ignite.lang.IgniteFuture; -import org.apache.ignite.ml.dlearn.DLearnContext; -import org.apache.ignite.ml.dlearn.DLearnPartitionFactory; -import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; -import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; -import org.apache.ignite.ml.math.functions.IgniteBiConsumer; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; - -/** - * Learning context based in Ignite Cache storage. - * - * @param

type of learning context partition - */ -public class CacheDLearnContext

implements DLearnContext

{ - /** Ignite instance. */ - private final Ignite ignite; - - /** Learning context cache name. */ - private final String learningCtxCacheName; - - /** Partition factory. */ - private final DLearnPartitionFactory

partFactory; - - /** Learning context id. */ - private final UUID learningCtxId; - - /** Names of caches which partitions are reserved during computations. */ - private final Collection cacheNames; - - /** - * Constructs a new instance of cache learning context. - * - * @param ignite Ignite instance - * @param learningCtxCacheName learning context cache name - * @param partFactory partition factory - * @param learningCtxId learning context id - * @param cacheNames names of caches which partitions are reserved during computations - */ - public CacheDLearnContext(Ignite ignite, String learningCtxCacheName, DLearnPartitionFactory

partFactory, - UUID learningCtxId, Collection cacheNames) { - this.ignite = ignite; - this.learningCtxCacheName = learningCtxCacheName; - this.partFactory = partFactory; - this.learningCtxId = learningCtxId; - this.cacheNames = cacheNames; - } - - /** {@inheritDoc} */ - public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer, R identity) { - ClusterGroup clusterGrp = ignite.cluster().forDataNodes(learningCtxCacheName); - - IgniteCache learningCtxCache = ignite.cache(learningCtxCacheName); - - Affinity affinity = ignite.affinity(learningCtxCacheName); - - List> futures = new ArrayList<>(affinity.partitions()); - - for (int part = 0; part < affinity.partitions(); part++) { - int currPart = part; - IgniteFuture fut = ignite.compute(clusterGrp).affinityCallAsync( - cacheNames, - part, - () -> { - DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, learningCtxId, currPart); - - P learningCtxPart = partFactory.createPartition(storage); - - return mapper.apply(learningCtxPart, currPart); - } - ); - futures.add(fut); - } - - List results = new ArrayList<>(affinity.partitions()); - for (IgniteFuture future : futures) - results.add(future.get()); - - return reduce(results, reducer, identity); - } - - /** {@inheritDoc} */ - @Override public void compute(IgniteBiConsumer mapper) { - ClusterGroup clusterGrp = ignite.cluster().forDataNodes(learningCtxCacheName); - - IgniteCache learningCtxCache = ignite.cache(learningCtxCacheName); - - Affinity affinity = ignite.affinity(learningCtxCacheName); - - List> futures = new ArrayList<>(affinity.partitions()); - - for (int part = 0; part < affinity.partitions(); part++) { - int currPart = part; - IgniteFuture fut = ignite.compute(clusterGrp).affinityRunAsync( - cacheNames, - part, - () -> { - DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, learningCtxId, currPart); - - P learningCtxPart = partFactory.createPartition(storage); - - mapper.accept(learningCtxPart, currPart); - } - ); - futures.add(fut); - } - - for (IgniteFuture future : futures) - future.get(); - } - - /** {@inheritDoc} */ - @Override public > C transform( - DLearnContextTransformer transformer) { - UUID newLearnCtxId = UUID.randomUUID(); - - compute((part, partIdx) -> { - IgniteCache learningCtxCache = ignite.cache(learningCtxCacheName); - DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, newLearnCtxId, partIdx); - - T newPart = transformer.createPartition(storage); - - transformer.transform(part, newPart); - }); - - DLearnContext newCtx = new CacheDLearnContext<>(ignite, learningCtxCacheName, transformer, newLearnCtxId, - Collections.singletonList(learningCtxCacheName)); - - return transformer.wrapContext(newCtx); - } - - /** - * Reduces results into a single final result. - * - * @param results results - * @param reducer reducer function - * @param identity identity - * @param type of result - * @return single final result - */ - private R reduce(Collection results, IgniteBinaryOperator reducer, R identity) { - R res = identity; - - for (R partRes : results) - res = reducer.apply(res, partRes); - - return res; - } - - /** {@inheritDoc} */ - @Override public void close() { - compute(this::closePartition); - } - - /** - * Closes partition. - * - * @param part partition to be closed - */ - private void closePartition(P part) { - try { - part.close(); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - /** */ - public String getLearningCtxCacheName() { - return learningCtxCacheName; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java deleted file mode 100644 index 62108784cbee4..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactory.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * 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.ml.dlearn.context.cache; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.UUID; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.ml.dlearn.DLearnContextFactory; -import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; -import org.apache.ignite.ml.dlearn.context.cache.utils.DLearnPartitionAffinityFunction; -import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; - -/** - * Factory produces cache learning context by extracting data from other cache. - * - * @param type of keys in upstream cache - * @param type of values in upstream values - */ -public class CacheDLearnContextFactory implements DLearnContextFactory> { - /** */ - private static final long serialVersionUID = 2903867793242785702L; - - /** Template for learning context cache name. */ - private static final String CONTEXT_CACHE_NAME = "%s_LEARNING_CONTEXT_%s"; - - /** Ignite instance. */ - private final Ignite ignite; - - /** Upstream cache with data. */ - private final IgniteCache upstreamCache; - - /** - * Constructs a new instance of cache learning context factory. - * - * @param ignite Ignite instance - * @param upstreamCache upstream cache - */ - public CacheDLearnContextFactory(Ignite ignite, IgniteCache upstreamCache) { - this.ignite = ignite; - this.upstreamCache = upstreamCache; - } - - /** {@inheritDoc} */ - @Override public CacheDLearnContext> createContext() { - CacheConfiguration learningCtxCacheCfg = new CacheConfiguration<>(); - learningCtxCacheCfg.setName(String.format(CONTEXT_CACHE_NAME, upstreamCache.getName(), UUID.randomUUID())); - learningCtxCacheCfg.setAffinity(createLearningContextCacheAffinityFunction()); - - IgniteCache learningCtxCache = ignite.createCache(learningCtxCacheCfg); - - Affinity affinity = ignite.affinity(upstreamCache.getName()); - UUID learningCtxId = UUID.randomUUID(); - - for (int partIdx = 0; partIdx < affinity.partitions(); partIdx++) { - DLearnPartitionStorage storage = new CacheDLearnPartitionStorage(learningCtxCache, learningCtxId, partIdx); - CacheDLearnPartition part = new CacheDLearnPartition<>(storage); - - part.setUpstreamCacheName(upstreamCache.getName()); - part.setPart(partIdx); - } - - return new CacheDLearnContext<>(ignite, learningCtxCache.getName(), CacheDLearnPartition::new, learningCtxId, - Arrays.asList(upstreamCache.getName(), learningCtxCache.getName())); - } - - /** - * Creates learning context cache affinity function based in upstream cache. This function retrieves current - * topology version and layout (partition-to-node map) of upstream cache corresponding to it, then applies this - * retrieved layout to create an affinity function for learning context cache. - * - * @return affinity function - */ - private DLearnPartitionAffinityFunction createLearningContextCacheAffinityFunction() { - Affinity affinity = ignite.affinity(upstreamCache.getName()); - - // tries to collect partition-to-node map and checks that topology version hasn't been changed during this - // process - List> initAssignment; - long topVer; - while (true) { - topVer = ignite.cluster().topologyVersion(); - - initAssignment = new ArrayList<>(affinity.partitions()); - - for (int part = 0; part < affinity.partitions(); part++) { - Collection nodes = affinity.mapPartitionToPrimaryAndBackups(part); - List nodeIds = new ArrayList<>(nodes.size()); - for (ClusterNode node : nodes) - nodeIds.add(node.id()); - initAssignment.add(nodeIds); - } - - // if topology version changed we need to try again - if (topVer == ignite.cluster().topologyVersion()) - break; - } - - return new DLearnPartitionAffinityFunction(initAssignment); - } -} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java deleted file mode 100644 index 1de06134613fc..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartition.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.ml.dlearn.context.cache; - -import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; - -/** - * D-learn partition which uses Ignite cache to keep data and produces as initial context partition by - * {@link CacheDLearnContextFactory}. - * - * @param type of keys - * @param type of values - */ -public class CacheDLearnPartition implements AutoCloseable { - /** Key of the upstream cache name values. */ - private static final String UPSTREAM_CACHE_NAME_KEY = "upstream_cache_name"; - - /** Key of the partition value. */ - private static final String PART_KEY = "part"; - - /** Partition storage. */ - private final DLearnPartitionStorage storage; - - /** - * Constructs a new instance of cache learning partition. - * - * @param storage partition storage - */ - public CacheDLearnPartition(DLearnPartitionStorage storage) { - this.storage = storage; - } - - /** */ - public void setUpstreamCacheName(String upstreamCacheName) { - storage.put(UPSTREAM_CACHE_NAME_KEY, upstreamCacheName); - } - - /** */ - public String getUpstreamCacheName() { - return storage.get(UPSTREAM_CACHE_NAME_KEY); - } - - /** */ - public void setPart(int part) { - storage.put(PART_KEY, part); - } - - /** */ - public int getPart() { - return storage.get(PART_KEY); - } - - /** - * Removes all data associated with the partition. - */ - @Override public void close() { - storage.remove(UPSTREAM_CACHE_NAME_KEY); - storage.remove(PART_KEY); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartitionStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartitionStorage.java deleted file mode 100644 index c113167293735..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnPartitionStorage.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * 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.ml.dlearn.context.cache; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.util.UUID; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; -import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; - -/** - * D-learn partition storage based on Ignite cache. - */ -public class CacheDLearnPartitionStorage implements DLearnPartitionStorage { - /** Learning context physical storage. */ - private final IgniteCache learningCtxCache; - - /** Learning context id. */ - private final UUID learningCtxId; - - /** Partition index. */ - private final int part; - - /** - * Constructs a new instance of cache learning partition storage. - * - * @param learningCtxCache learning context physical storage - * @param learningCtxId learning context id - * @param part partition index - */ - public CacheDLearnPartitionStorage(IgniteCache learningCtxCache, UUID learningCtxId, int part) { - this.learningCtxCache = learningCtxCache; - this.learningCtxId = learningCtxId; - this.part = part; - } - - /** {@inheritDoc} */ - @Override public void put(String key, T val) { - learningCtxCache.put(new DLearnContextPartitionKey(part, learningCtxId, key), serialize(val)); - } - - /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - @Override public T get(String key) { - return (T) deserialize(learningCtxCache.localPeek(new DLearnContextPartitionKey(part, learningCtxId, key))); - } - - /** {@inheritDoc} */ - @Override public void remove(String key) { - learningCtxCache.remove(new DLearnContextPartitionKey(part, learningCtxId, key)); - } - - /** - * Serializes specified object into byte array. - * - * @param obj object - * @return byte arrays representing serialized object - */ - private byte[] serialize(Object obj) { - if (obj == null) - return null; - - try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ObjectOutputStream oos = new ObjectOutputStream(baos)) { - oos.writeObject(obj); - - return baos.toByteArray(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - /** - * Deserializes object from specified byte array. - * - * @param arr byte array representing serialized object - * @return object - */ - private Object deserialize(byte[] arr) { - if (arr == null) - return null; - - try (ByteArrayInputStream bais = new ByteArrayInputStream(arr); - ObjectInputStream ois = new ObjectInputStream(bais)) { - - return ois.readObject(); - } - catch (IOException | ClassNotFoundException e) { - throw new RuntimeException(e); - } - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/package-info.java deleted file mode 100644 index 149e694fae31a..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Root package for distributed learning implementations based on Ignite Cache. - */ -package org.apache.ignite.ml.dlearn.context.cache; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunction.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunction.java deleted file mode 100644 index edca19678f5cb..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunction.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * 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.ml.dlearn.context.cache.utils; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.UUID; -import org.apache.ignite.binary.BinaryObject; -import org.apache.ignite.cache.affinity.AffinityFunction; -import org.apache.ignite.cache.affinity.AffinityFunctionContext; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; - -/** - * This affinity function is used to identify a partition by key and node to place the partition. This function is - * initialized with {@link #initAssignment} parameter which contains information about upstream cache distribution - * across the cluster. This information allows function to place context partitions on the same nodes as partitions - * of the upstream cache. Be aware that this affinity functions supports only {@link DLearnContextPartitionKey} keys. - */ -public class DLearnPartitionAffinityFunction implements AffinityFunction { - /** */ - private static final long serialVersionUID = 7735390384525189270L; - - /** - * Initial distribution of the partitions (copy of upstream cache partitions distribution). - */ - private final List> initAssignment; - - /** - * Creates new instance of d-learn partition affinity function initialized with initial distribution. - * - * @param initAssignment initial distribution of the partitions (copy of upstream cache partitions distribution) - */ - public DLearnPartitionAffinityFunction(List> initAssignment) { - this.initAssignment = initAssignment; - } - - /** {@inheritDoc} */ - @Override public void reset() { - // do nothing - } - - /** {@inheritDoc} */ - @Override public int partitions() { - return initAssignment.size(); - } - - /** {@inheritDoc} */ - @Override public int partition(Object key) { - BinaryObject bo = (BinaryObject) key; - DLearnContextPartitionKey datasetPartKey = bo.deserialize(); - return datasetPartKey.getPart(); - } - - /** {@inheritDoc} */ - @Override public List> assignPartitions(AffinityFunctionContext affCtx) { - List> assignment = new ArrayList<>(initAssignment.size()); - - Map topSnapshotIdx = new HashMap<>(); - List topSnapshot = affCtx.currentTopologySnapshot(); - - for (ClusterNode node : topSnapshot) - topSnapshotIdx.put(node.id(), node); - - for (int part = 0; part < initAssignment.size(); part++) { - List partNodes = new ArrayList<>(1 + affCtx.backups()); - List partNodeIds = initAssignment.get(part); - - // looking for a primary node - ClusterNode primaryPartNode = null; - for (UUID partNodeId : partNodeIds) { - primaryPartNode = topSnapshotIdx.get(partNodeId); - if (primaryPartNode != null) - break; - } - - if (primaryPartNode == null) { - Random random = new Random(); - int nodeIdx = random.nextInt(topSnapshot.size()); - primaryPartNode = topSnapshot.get(nodeIdx); - } - partNodes.add(primaryPartNode); - - assignment.add(partNodes); - } - - return assignment; - } - - /** {@inheritDoc} */ - @Override public void removeNode(UUID nodeId) { - // do nothing - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java deleted file mode 100644 index a13b2f4d84389..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContext.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * 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.ml.dlearn.context.local; - -import java.util.Map; -import java.util.UUID; -import org.apache.ignite.ml.dlearn.DLearnContext; -import org.apache.ignite.ml.dlearn.DLearnPartitionFactory; -import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; -import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; -import org.apache.ignite.ml.math.functions.IgniteBiConsumer; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; - -/** - * Learning context based on a local on-heap storage. - * - * @param

type of learning context partition - */ -public class LocalDLearnContext

implements DLearnContext

{ - /** Physical storage of the learning context data. */ - private final Map learningCtxMap; - - /** Learning context partition factory. */ - private final DLearnPartitionFactory

partFactory; - - /** Learning context id. */ - private final UUID learningCtxId; - - /** Number of partitions. */ - private final int partitions; - - /** - * Constructs a new instance of local learning context. - * - * @param learningCtxMap physical storage of the learning context data - * @param partFactory learning context partition factory - * @param learningCtxId learning context id - * @param partitions number of partitions - */ - public LocalDLearnContext(Map learningCtxMap, - DLearnPartitionFactory

partFactory, UUID learningCtxId, int partitions) { - this.learningCtxMap = learningCtxMap; - this.partFactory = partFactory; - this.learningCtxId = learningCtxId; - this.partitions = partitions; - } - - /** {@inheritDoc} */ - @Override public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer, - R identity) { - R res = identity; - for (int partIdx = 0; partIdx < partitions; partIdx++) { - DLearnPartitionStorage storage = new LocalDLearnPartitionStorage(learningCtxMap, learningCtxId, partIdx); - - P part = partFactory.createPartition(storage); - R partRes = mapper.apply(part, partIdx); - - res = reducer.apply(res, partRes); - } - return res; - } - - /** {@inheritDoc} */ - @Override public void compute(IgniteBiConsumer mapper) { - for (int partIdx = 0; partIdx < partitions; partIdx++) { - DLearnPartitionStorage storage = new LocalDLearnPartitionStorage(learningCtxMap, learningCtxId, partIdx); - - P part = partFactory.createPartition(storage); - - mapper.accept(part, partIdx); - } - } - - /** {@inheritDoc} */ - @Override public > C transform( - DLearnContextTransformer transformer) { - UUID newLearnCtxId = UUID.randomUUID(); - - compute((part, partIdx) -> { - DLearnPartitionStorage newStorage = new LocalDLearnPartitionStorage(learningCtxMap, newLearnCtxId, partIdx); - - T newPart = transformer.createPartition(newStorage); - - transformer.transform(part, newPart); - }); - - DLearnContext newCtx = new LocalDLearnContext<>(learningCtxMap, transformer, newLearnCtxId, partitions); - - return transformer.wrapContext(newCtx); - } - - /** {@inheritDoc} */ - @Override public void close() { - compute(this::closePartition); - } - - /** - * Closes partition. - * - * @param part partition to be closed - */ - private void closePartition(P part) { - try { - part.close(); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - /** */ - public Map getLearningCtxMap() { - return learningCtxMap; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java deleted file mode 100644 index c3e4de4929bd6..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextFactory.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * 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.ml.dlearn.context.local; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import org.apache.ignite.ml.dlearn.DLearnContextFactory; -import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; -import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; - -/** - * Factory produces local learning context by extracting data from local list of values. - * - * @param type of upstream values - */ -public class LocalDLearnContextFactory implements DLearnContextFactory> { - /** */ - private static final long serialVersionUID = -7614441997952907675L; - - /** Upstream data. */ - private final Map data; - - /** Number of partitions. */ - private final int partitions; - - /** - * Constructs a new instance of learning context factory. - * - * @param data upstream data - * @param partitions number of partitions - */ - public LocalDLearnContextFactory(Map data, int partitions) { - this.data = data; - this.partitions = partitions; - } - - /** {@inheritDoc} */ - @Override public LocalDLearnContext> createContext() { - Map learningCtxMap = new HashMap<>(); - - UUID learningCtxId = UUID.randomUUID(); - - int partSize = data.size() / partitions; - - // loads data into learning context partitions - List keys = new ArrayList<>(data.keySet()); - - for (int partIdx = 0; partIdx < partitions; partIdx++) { - Map partData = new HashMap<>(); - - for (int j = partIdx * partSize; j < (partIdx + 1) * partSize && j < data.size(); j++) - partData.put(keys.get(j), data.get(keys.get(j))); - - DLearnPartitionStorage storage = new LocalDLearnPartitionStorage(learningCtxMap, learningCtxId, partIdx); - LocalDLearnPartition part = new LocalDLearnPartition<>(storage); - - part.setPartData(partData); - } - - return new LocalDLearnContext<>(learningCtxMap, LocalDLearnPartition::new, learningCtxId, partitions); - } -} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java deleted file mode 100644 index c6673e93df742..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartition.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.ml.dlearn.context.local; - -import java.util.Map; -import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; - -/** - * D-learn partition which uses local on-heap hash map to keep data and produced as initial context partition by - * {@link LocalDLearnContextFactory}. - * - * @param type of keys - * @param type of values - */ -public class LocalDLearnPartition implements AutoCloseable { - /** Key of partition data values. */ - private static final String PART_DATA_KEY = "part_data"; - - /** Partition storage. */ - private final DLearnPartitionStorage storage; - - /** - * Constructs a new instance of local learning partition. - * - * @param storage partition storage - */ - public LocalDLearnPartition(DLearnPartitionStorage storage) { - this.storage = storage; - } - - /** */ - public Map getPartData() { - return storage.get(PART_DATA_KEY); - } - - /** */ - public void setPartData(Map partData) { - storage.put(PART_DATA_KEY, partData); - } - - /** - * Removes all data associated with the partition. - */ - @Override public void close() { - storage.remove(PART_DATA_KEY); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartitionStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartitionStorage.java deleted file mode 100644 index 175e5fe1bd091..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnPartitionStorage.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * 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.ml.dlearn.context.local; - -import java.util.Map; -import java.util.UUID; -import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; -import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; - -/** - * D-learn partition storage based on on-heap hash map for local processing and tests. Doesn't require Ignite cluster - * to work. - */ -public class LocalDLearnPartitionStorage implements DLearnPartitionStorage { - /** Learning context physical storage. */ - private final Map learningCtxMap; - - /** Learning context id. */ - private final UUID learningCtxId; - - /** Partition index. */ - private final int part; - - /** - * Constructs a new instance of local learning partition storage. - * - * @param learningCtxMap learning context physical storage - * @param learningCtxId learning context id - * @param part partition index - */ - public LocalDLearnPartitionStorage( - Map learningCtxMap, UUID learningCtxId, int part) { - this.learningCtxMap = learningCtxMap; - this.learningCtxId = learningCtxId; - this.part = part; - } - - /** {@inheritDoc} */ - @Override public void put(String key, T val) { - learningCtxMap.put(new DLearnContextPartitionKey(part, learningCtxId, key), val); - } - - /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - @Override public T get(String key) { - return (T)learningCtxMap.get(new DLearnContextPartitionKey(part, learningCtxId, key)); - } - - /** {@inheritDoc} */ - @Override public void remove(String key) { - learningCtxMap.remove(new DLearnContextPartitionKey(part, learningCtxId, key)); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/package-info.java deleted file mode 100644 index 35a3554acd741..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/local/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Root package for distributed learning implementations based on local storage. - */ -package org.apache.ignite.ml.dlearn.context.local; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/package-info.java deleted file mode 100644 index fbab0bf7c3184..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Root package for underlying d-learn implementations (based on cache, local storage, etc...). - */ -package org.apache.ignite.ml.dlearn.context; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformer.java deleted file mode 100644 index 05ea3a9f2683a..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformer.java +++ /dev/null @@ -1,34 +0,0 @@ -package org.apache.ignite.ml.dlearn.context.transformer; - -import org.apache.ignite.ml.dlearn.DLearnContext; -import org.apache.ignite.ml.dlearn.DLearnPartitionFactory; - -/** - * Transformer which allow to transform one learning context into another. Transformation mean that new d-learn - * partitions will be created from old partitions and saved in the same underlying storage, but with a new learning - * context id. New partitions will be containing a new data required to provide new API as a new learning context. - * - * All generic transformers are aggregated in {@link DLearnContextTransformers}. - * - * @param

type of an initial partition - * @param type of a new partition - * @param type of a new learning context - */ -public interface DLearnContextTransformer

> - extends DLearnPartitionFactory { - /** - * Copies required data from old partition into new one. All needed transformations are allowed. - * - * @param oldPart old (initial) partition - * @param newPart new partition - */ - public void transform(P oldPart, T newPart); - - /** - * Wraps learning context to provide partition-specific API. - * - * @param ctx context - * @return wrapped context - */ - public C wrapContext(DLearnContext ctx); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java deleted file mode 100644 index 35691f904f35b..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/DLearnContextTransformers.java +++ /dev/null @@ -1,77 +0,0 @@ -package org.apache.ignite.ml.dlearn.context.transformer; - -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContextFactory; -import org.apache.ignite.ml.dlearn.context.local.LocalDLearnContextFactory; -import org.apache.ignite.ml.dlearn.context.transformer.cache.CacheDatasetDLearnPartitionTransformer; -import org.apache.ignite.ml.dlearn.context.transformer.cache.CacheLabeledDatasetDLearnPartitionTransformer; -import org.apache.ignite.ml.dlearn.context.transformer.local.LocalDatasetDLearnPartitionTransformer; -import org.apache.ignite.ml.dlearn.context.transformer.local.LocalLabeledDatasetDLearnPartitionTransformer; -import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; -import org.apache.ignite.ml.dlearn.dataset.DLearnLabeledDataset; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; - -/** - * Aggregator which allows to find desired transformer from one learning context into another. This class doesn't - * introduce a new functionality, but helps to work efficiently with existing transformers. - */ -public class DLearnContextTransformers { - /** - * Creates a transformer which accepts cache learning context (produced by {@link CacheDLearnContextFactory}) and - * constructs {@link DLearnDataset}. - * - * @param featureExtractor feature extractor - * @param type of keys in cache learning context - * @param type of values in cache learning context - * @return transformer - */ - public static CacheDatasetDLearnPartitionTransformer cacheToDataset( - IgniteBiFunction featureExtractor) { - return new CacheDatasetDLearnPartitionTransformer<>(featureExtractor); - } - - /** - * Creates a transformer which accepts cache learning context (produced by {@link CacheDLearnContextFactory}) and - * constructs {@link DLearnLabeledDataset}. - * - * @param featureExtractor feature extractor - * @param lbExtractor label extractor - * @param type of keys in cache learning context - * @param type of values in cache learning context - * @param type of label - * @return transformer - */ - public static CacheLabeledDatasetDLearnPartitionTransformer cacheToLabeledDataset( - IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { - return new CacheLabeledDatasetDLearnPartitionTransformer<>(featureExtractor, lbExtractor); - } - - /** - * Creates a transformer which accepts local learning context (produced by {@link LocalDLearnContextFactory}) and - * constructs {@link DLearnDataset}. - * - * @param featureExtractor feature extractor - * @param type of keys in local learning context - * @param type of values in local learning context - * @return transformer - */ - public static LocalDatasetDLearnPartitionTransformer localToDataset( - IgniteBiFunction featureExtractor) { - return new LocalDatasetDLearnPartitionTransformer<>(featureExtractor); - } - - /** - * Creates a transformer which accepts cache learning context (produced by {@link LocalDLearnContextFactory}) and - * constructs {@link DLearnLabeledDataset}. - * - * @param featureExtractor feature extractor - * @param lbExtractor label extractor - * @param type of keys in local learning context - * @param type of values in local learning context - * @param type of label - * @return transformer - */ - public static LocalLabeledDatasetDLearnPartitionTransformer localToLabeledDataset( - IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { - return new LocalLabeledDatasetDLearnPartitionTransformer<>(featureExtractor, lbExtractor); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java deleted file mode 100644 index 53a4192cddbfb..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheDatasetDLearnPartitionTransformer.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * 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.ml.dlearn.context.transformer.cache; - -import java.util.ArrayList; -import java.util.List; -import javax.cache.Cache; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.Ignition; -import org.apache.ignite.cache.query.QueryCursor; -import org.apache.ignite.cache.query.ScanQuery; -import org.apache.ignite.ml.dlearn.DLearnContext; -import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContextFactory; -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; -import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; -import org.apache.ignite.ml.dlearn.dataset.part.DLeanDatasetPartition; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; - -/** - * Creates a transformer which accepts cache learning context (produced by {@link CacheDLearnContextFactory}) and - * constructs {@link DLearnDataset}. - * - * @param type of keys in cache learning context - * @param type of values in cache learning context - */ -public class CacheDatasetDLearnPartitionTransformer - implements DLearnContextTransformer, DLeanDatasetPartition, - DLearnDataset> { - /** */ - private static final long serialVersionUID = -7398727071330763144L; - - /** Feature extractor. */ - private final IgniteBiFunction featureExtractor; - - /** - * Constructs a new instance of cache to dataset partition transformer. - * - * @param featureExtractor feature extractor - */ - public CacheDatasetDLearnPartitionTransformer(IgniteBiFunction featureExtractor) { - this.featureExtractor = featureExtractor; - } - - /** {@inheritDoc} */ - @Override public void transform(CacheDLearnPartition oldPart, DLeanDatasetPartition newPart) { - List> partData = queryPartDataIntoList(oldPart); - - double[] features = null; - int m = partData.size(), n = 0; - - for (int i = 0; i < partData.size(); i++) { - Cache.Entry entry = partData.get(i); - double[] rowFeatures = featureExtractor.apply(entry.getKey(), entry.getValue()); - - if (i == 0) { - n = rowFeatures.length; - features = new double[m * n]; - } - - if (rowFeatures.length != n) - throw new IllegalStateException(); - - for (int j = 0; j < rowFeatures.length; j++) - features[j * m + i] = rowFeatures[j]; - } - - newPart.setFeatures(features); - newPart.setRows(m); - } - - /** {@inheritDoc} */ - @Override public DLearnDataset wrapContext(DLearnContext ctx) { - return new DLearnDataset<>(ctx); - } - - /** {@inheritDoc} */ - @Override public DLeanDatasetPartition createPartition(DLearnPartitionStorage storage) { - return new DLeanDatasetPartition(storage); - } - - /** - * Retrieves local partition data from the cache via {@link ScanQuery} and collects it into list. - * - * @param oldPart partition - * @return list of cache entries - */ - private List> queryPartDataIntoList(CacheDLearnPartition oldPart) { - List> partData = new ArrayList<>(); - try (QueryCursor> cursor = queryPartData(oldPart)) { - for (Cache.Entry entry : cursor) - partData.add(entry); - return partData; - } - } - - /** - * Retrieves local partition data from the cache via {@link ScanQuery} and returns cursor. - * - * @param oldPart partition - * @return cursor - */ - private QueryCursor> queryPartData(CacheDLearnPartition oldPart) { - Ignite ignite = Ignition.localIgnite(); - IgniteCache upstreamCache = ignite.cache(oldPart.getUpstreamCacheName()); - - ScanQuery qry = new ScanQuery<>(); - qry.setPartition(oldPart.getPart()); - - return upstreamCache.query(qry); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java deleted file mode 100644 index 720fceb3fa22b..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/CacheLabeledDatasetDLearnPartitionTransformer.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * 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.ml.dlearn.context.transformer.cache; - -import java.util.ArrayList; -import java.util.List; -import javax.cache.Cache; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.Ignition; -import org.apache.ignite.cache.query.ScanQuery; -import org.apache.ignite.ml.dlearn.DLearnContext; -import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContextFactory; -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; -import org.apache.ignite.ml.dlearn.dataset.DLearnLabeledDataset; -import org.apache.ignite.ml.dlearn.dataset.part.DLearnLabeledDatasetPartition; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; - -/** - * Creates a transformer which accepts cache learning context (produced by {@link CacheDLearnContextFactory}) and - * constructs {@link DLearnLabeledDataset}. - * - * @param type of keys in cache learning context - * @param type of values in cache learning context - * @param type of label - */ -public class CacheLabeledDatasetDLearnPartitionTransformer - implements DLearnContextTransformer, DLearnLabeledDatasetPartition, - DLearnLabeledDataset> { - /** */ - private static final long serialVersionUID = 3479218902890029731L; - - /** Feature extractor. */ - private final IgniteBiFunction featureExtractor; - - /** Label extractor. */ - private final IgniteBiFunction lbExtractor; - - /** - * Constructs a new instance of cache to labeled dataset partition transformer. - * - * @param featureExtractor feature extractor - * @param lbExtractor label extractor - */ - public CacheLabeledDatasetDLearnPartitionTransformer( - IgniteBiFunction featureExtractor, - IgniteBiFunction lbExtractor) { - this.featureExtractor = featureExtractor; - this.lbExtractor = lbExtractor; - } - - /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - @Override public void transform(CacheDLearnPartition oldPart, DLearnLabeledDatasetPartition newPart) { - List> partData = queryPartDataIntoList(oldPart); - - int m = partData.size(), n = 0; - double[] features = null; - L[] labels = (L[])new Object[m]; - - for (int i = 0; i < partData.size(); i++) { - Cache.Entry entry = partData.get(i); - double[] rowFeatures = featureExtractor.apply(entry.getKey(), entry.getValue()); - labels[i] = lbExtractor.apply(entry.getKey(), entry.getValue()); - - if (i == 0) { - n = rowFeatures.length; - features = new double[m * n]; - } - - if (rowFeatures.length != n) - throw new IllegalStateException(); - - for (int j = 0; j < rowFeatures.length; j++) - features[j * m + i] = rowFeatures[j]; - } - - newPart.setFeatures(features); - newPart.setRows(m); - newPart.setLabels(labels); - } - - /** {@inheritDoc} */ - @Override public DLearnLabeledDataset wrapContext(DLearnContext> ctx) { - return new DLearnLabeledDataset<>(ctx); - } - - /** {@inheritDoc} */ - @Override public DLearnLabeledDatasetPartition createPartition(DLearnPartitionStorage storage) { - return new DLearnLabeledDatasetPartition<>(storage); - } - - /** - * Retrieves local partition data from the cache via {@link ScanQuery} and collects it into list. - * - * @param oldPart partition - * @return list of cache entries - */ - private List> queryPartDataIntoList(CacheDLearnPartition oldPart) { - List> partData = new ArrayList<>(); - for (Cache.Entry entry : queryPartData(oldPart)) - partData.add(entry); - return partData; - } - - /** - * Retrieves local partition data from the cache via {@link ScanQuery} and returns cursor. - * - * @param oldPart partition - * @return cursor - */ - private Iterable> queryPartData(CacheDLearnPartition oldPart) { - Ignite ignite = Ignition.localIgnite(); - IgniteCache upstreamCache = ignite.cache(oldPart.getUpstreamCacheName()); - - ScanQuery qry = new ScanQuery<>(); - qry.setPartition(oldPart.getPart()); - - return upstreamCache.query(qry); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/package-info.java deleted file mode 100644 index 31025da3914ab..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/cache/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Contains transformers which allows to transform cache learning context into some another contexts. - */ -package org.apache.ignite.ml.dlearn.context.transformer.cache; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java deleted file mode 100644 index 1c70a2b38b05a..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalDatasetDLearnPartitionTransformer.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * 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.ml.dlearn.context.transformer.local; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import org.apache.ignite.ml.dlearn.DLearnContext; -import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; -import org.apache.ignite.ml.dlearn.context.local.LocalDLearnContextFactory; -import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; -import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; -import org.apache.ignite.ml.dlearn.dataset.part.DLeanDatasetPartition; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; - -/** - * Creates a transformer which accepts local learning context (produced by {@link LocalDLearnContextFactory}) and - * constructs {@link DLearnDataset}. - * - * @param type of keys in local learning context - * @param type of values in local learning context - */ -public class LocalDatasetDLearnPartitionTransformer - implements DLearnContextTransformer, DLeanDatasetPartition, - DLearnDataset> { - /** */ - private static final long serialVersionUID = -7567051002880704559L; - - /** Feature extractor. */ - private final IgniteBiFunction featureExtractor; - - /** - * Constructs new instance of local to dataset partition transformer. - * - * @param featureExtractor feature extractor - */ - public LocalDatasetDLearnPartitionTransformer(IgniteBiFunction featureExtractor) { - this.featureExtractor = featureExtractor; - } - - /** {@inheritDoc} */ - @Override public void transform(LocalDLearnPartition oldPart, DLeanDatasetPartition newPart) { - Map partData = oldPart.getPartData(); - - if (partData != null && !partData.isEmpty()) { - double[] features = null; - int m = partData.size(), n = 0; - - List keys = new ArrayList<>(partData.keySet()); - - for (int i = 0; i < partData.size(); i++) { - K key = keys.get(i); - double[] rowFeatures = featureExtractor.apply(key, partData.get(key)); - - if (i == 0) { - n = rowFeatures.length; - features = new double[m * n]; - } - - if (rowFeatures.length != n) - throw new IllegalStateException(); - - for (int j = 0; j < rowFeatures.length; j++) - features[j * m + i] = rowFeatures[j]; - } - - newPart.setFeatures(features); - newPart.setRows(m); - } - } - - /** {@inheritDoc} */ - @Override public DLearnDataset wrapContext(DLearnContext ctx) { - return new DLearnDataset<>(ctx); - } - - /** {@inheritDoc} */ - @Override public DLeanDatasetPartition createPartition(DLearnPartitionStorage storage) { - return new DLeanDatasetPartition(storage); - } -} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java deleted file mode 100644 index 7bd41b1bc9eb7..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/LocalLabeledDatasetDLearnPartitionTransformer.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * 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.ml.dlearn.context.transformer.local; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import org.apache.ignite.ml.dlearn.DLearnContext; -import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; -import org.apache.ignite.ml.dlearn.context.local.LocalDLearnContextFactory; -import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; -import org.apache.ignite.ml.dlearn.dataset.DLearnLabeledDataset; -import org.apache.ignite.ml.dlearn.dataset.part.DLearnLabeledDatasetPartition; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; - -/** - * Creates a transformer which accepts cache learning context (produced by {@link LocalDLearnContextFactory}) and - * constructs {@link DLearnLabeledDataset}. - * - * @param type of keys in local learning context - * @param type of values in local learning context - * @param type of label - */ -public class LocalLabeledDatasetDLearnPartitionTransformer - implements DLearnContextTransformer, DLearnLabeledDatasetPartition, - DLearnLabeledDataset> { - /** */ - private static final long serialVersionUID = -8438445094768312331L; - - /** Feature extractor. */ - private final IgniteBiFunction featureExtractor; - - /** Label extractor. */ - private final IgniteBiFunction lbExtractor; - - /** - * Creates new instance of local to labeled dataset transformer. - * - * @param featureExtractor feature extractor - * @param lbExtractor label extractor - */ - public LocalLabeledDatasetDLearnPartitionTransformer(IgniteBiFunction featureExtractor, - IgniteBiFunction lbExtractor) { - this.featureExtractor = featureExtractor; - this.lbExtractor = lbExtractor; - } - - /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - @Override public void transform(LocalDLearnPartition oldPart, DLearnLabeledDatasetPartition newPart) { - Map partData = oldPart.getPartData(); - - if (partData != null && !partData.isEmpty()) { - double[] features = null; - int m = partData.size(), n = 0; - - List keys = new ArrayList<>(partData.keySet()); - - for (int i = 0; i < partData.size(); i++) { - K key = keys.get(i); - double[] rowFeatures = featureExtractor.apply(key, partData.get(key)); - - if (i == 0) { - n = rowFeatures.length; - features = new double[m * n]; - } - - if (rowFeatures.length != n) - throw new IllegalStateException(); - - for (int j = 0; j < rowFeatures.length; j++) - features[j * m + i] = rowFeatures[j]; - } - - newPart.setFeatures(features); - newPart.setRows(m); - - L[] labels = (L[])new Object[partData.size()]; - - for (int i = 0; i < partData.size(); i++) { - K key = keys.get(i); - labels[i] = lbExtractor.apply(key, partData.get(key)); - } - - newPart.setLabels(labels); - } - } - - /** {@inheritDoc} */ - @Override public DLearnLabeledDataset wrapContext(DLearnContext> ctx) { - return new DLearnLabeledDataset<>(ctx); - } - - /** {@inheritDoc} */ - @Override public DLearnLabeledDatasetPartition createPartition(DLearnPartitionStorage storage) { - return new DLearnLabeledDatasetPartition<>(storage); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/package-info.java deleted file mode 100644 index 63b895e7ed420..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/local/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Contains transformers which allows to transform local learning context into some another contexts. - */ -package org.apache.ignite.ml.dlearn.context.transformer.local; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/package-info.java deleted file mode 100644 index a80edfef99b6c..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/context/transformer/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Contains different transformers which allows to transform one learning context into another. - */ -package org.apache.ignite.ml.dlearn.context.transformer; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java deleted file mode 100644 index 9c249ebdf8d62..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/AbstractDLearnContextWrapper.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * 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.ml.dlearn.dataset; - -import org.apache.ignite.ml.dlearn.DLearnContext; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; -import org.apache.ignite.ml.math.functions.IgniteBiConsumer; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; - -/** - * Wrapper of a learning context which allows to introduce new context-specific methods based on base {@code compute()} - * functionality. - * - * @param

type of a d-learn partition - */ -public class AbstractDLearnContextWrapper

implements DLearnContext

{ - /** - * Delegate which actually performs base functions like {@code compute()}, {@code transform()} and {@code close()}. - */ - protected final DLearnContext

delegate; - - /** - * Constructs a new instance of context wrapper which delegates base operations to {@code delegate}. - * - * @param delegate delegate - */ - public AbstractDLearnContextWrapper(DLearnContext

delegate) { - this.delegate = delegate; - } - - /** {@inheritDoc} */ - @Override public R compute(IgniteBiFunction mapper, IgniteBinaryOperator reducer, - R identity) { - return delegate.compute(mapper, reducer, identity); - } - - /** {@inheritDoc} */ - @Override public void compute(IgniteBiConsumer mapper) { - delegate.compute(mapper); - } - - /** {@inheritDoc} */ - @Override public > C transform( - DLearnContextTransformer transformer) { - return delegate.transform(transformer); - } - - /** {@inheritDoc} */ - @Override public void close() { - delegate.close(); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnDataset.java deleted file mode 100644 index 03b2ae9d35431..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnDataset.java +++ /dev/null @@ -1,206 +0,0 @@ -/* - * 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.ml.dlearn.dataset; - -import com.github.fommil.netlib.BLAS; -import org.apache.ignite.ml.dlearn.DLearnContext; -import org.apache.ignite.ml.dlearn.dataset.part.DLeanDatasetPartition; - -/** - * Dataset provides API to work with dataset. - * - * @param

type of a d-learn partition - */ -public class DLearnDataset

extends AbstractDLearnContextWrapper

{ - /** BLAS (Basic Linear Algebra Subprograms) instance. */ - private static final BLAS blas = BLAS.getInstance(); - - /** {@inheritDoc} */ - public DLearnDataset(DLearnContext

delegate) { - super(delegate); - } - - /** - * Calculates mean value by given columns. - * - * @param cols columns - * @return mean values - */ - public double[] mean(int[] cols) { - ValueWithCount res = delegate.compute((part, partIdx) -> { - double[] features = part.getFeatures(); - int m = part.getRows(); - - double[] y = new double[cols.length]; - - for (int i = 0; i < cols.length; i++) - for (int j = cols[i] * m; j < (cols[i] + 1) * m; j++) - y[i] += features[j]; - - return new ValueWithCount<>(y, m); - }, (a, b) -> a == null ? b : b == null ? a : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); - - if (res != null) { - blas.dscal(res.val.length, 1.0 / res.cnt, res.val, 1); - return res.val; - } - - return null; - } - - /** - * Calculates mean value by the given column. - * - * @param col column - * @return mean value - */ - public double mean(int col) { - return mean(new int[]{col})[0]; - } - - /** - * Calculates standard deviation by given columns. - * - * @param cols columns - * @return standard deviations - */ - public double[] std(int[] cols) { - double[] mean = mean(cols); - ValueWithCount res = delegate.compute(part -> { - double[] features = part.getFeatures(); - int m = part.getRows(); - - double[] y = new double[cols.length]; - - for (int i = 0; i < cols.length; i++) - for (int j = cols[i] * m; j < (cols[i] + 1) * m; j++) - y[i] += Math.pow(features[j] - mean[cols[i]], 2); - - return new ValueWithCount<>(y, m); - }, (a, b) -> a == null ? b : b == null ? a : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); - - if (res != null) { - blas.dscal(res.val.length, 1.0 / res.cnt, res.val, 1); - for (int i = 0; i < res.val.length; i++) - res.val[i] = Math.sqrt(res.val[i]); - return res.val; - } - - return null; - } - - /** - * Calculates standard deviation by the given column. - * - * @param col column - * @return standard deviation - */ - public double std(int col) { - return std(new int[]{col})[0]; - } - - /** - * Calculates covariance matrix by given columns. - * - * @param cols columns - * @return covariance matrix - */ - public double[][] cov(int[] cols) { - double[] mean = mean(cols); - ValueWithCount res = delegate.compute(part -> { - double[] features = part.getFeatures(); - int m = part.getRows(); - - double[][] y = new double[cols.length][cols.length]; - - for (int i = 0; i < cols.length; i++) - for (int j = 0; j < cols.length; j++) { - int firstCol = cols[i]; - int secondCol = cols[j]; - - for (int k = 0; k < m; k++) { - double firstVal = features[m * firstCol + k]; - double secondVal = features[m * secondCol + k]; - y[i][j] += ((firstVal - mean[firstCol]) * (secondVal - mean[secondCol])); - } - } - - return new ValueWithCount<>(y, m); - }, (a, b) -> a == null ? b : b == null ? a : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); - - return res != null ? scale(res.val, 1.0 / res.cnt) : null; - } - - /** - * Calculates correlation matrix by given columns. - * - * @param cols columns - * @return correlation matrix - */ - public double[][] corr(int[] cols) { - double[][] cov = cov(cols); - double[] std = std(cols); - - for (int i = 0; i < cov.length; i++) - for (int j = 0; j < cov[0].length; j++) - cov[i][j] /= (std[i]*std[j]); - - return cov; - } - - /** */ - private static double[] sum(double[] a, double[] b) { - for (int i = 0; i < a.length; i++) - a[i] += b[i]; - - return a; - } - - /** */ - private static double[][] sum(double[][] a, double[][] b) { - for (int i = 0; i < a.length; i++) - for (int j = 0; j < a[i].length; j++) - a[i][j] += b[i][j]; - - return a; - } - - /** */ - private static double[][] scale(double[][] a, double alpha) { - for (int i = 0; i < a.length; i++) - for (int j = 0; j < a[i].length; j++) - a[i][j] *= alpha; - - return a; - } - - /** */ - private static class ValueWithCount { - /** */ - private final V val; - - /** */ - private final int cnt; - - /** */ - ValueWithCount(V val, int cnt) { - this.val = val; - this.cnt = cnt; - } - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnLabeledDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnLabeledDataset.java deleted file mode 100644 index d25179b158b00..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/DLearnLabeledDataset.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.ml.dlearn.dataset; - -import org.apache.ignite.ml.dlearn.DLearnContext; -import org.apache.ignite.ml.dlearn.dataset.part.DLearnLabeledDatasetPartition; - -/** - * Dataset provides API to work with labeled dataset. - * - * @param type of a label - */ -public class DLearnLabeledDataset extends DLearnDataset> { - /** {@inheritDoc} */ - public DLearnLabeledDataset( - DLearnContext> delegate) { - super(delegate); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/package-info.java deleted file mode 100644 index 28b60f7120aea..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Root package for API classes used in algorithms based in d-learn framework. - */ -package org.apache.ignite.ml.dlearn.dataset; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLeanDatasetPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLeanDatasetPartition.java deleted file mode 100644 index 5270298427a66..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLeanDatasetPartition.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * 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.ml.dlearn.dataset.part; - -import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; - -/** - * Interface which provides simple dataset API which allows to get or set an underlying feature matrix in flat format. - */ -public class DLeanDatasetPartition implements AutoCloseable { - /** Key of feature values. */ - private static final String FEATURES_KEY = "features"; - - /** Key of row values. */ - private static final String ROWS_KEY = "rows"; - - /** Learning partition storage. */ - private final DLearnPartitionStorage storage; - - /** - * Creates a new instance of learning dataset partition. - * - * @param storage learning partition storage - */ - public DLeanDatasetPartition(DLearnPartitionStorage storage) { - this.storage = storage; - } - - /** - * Sets matrix of features in flat format. - * - * @param features matrix of features in flat format - */ - public void setFeatures(double[] features) { - storage.put(FEATURES_KEY, features); - } - - /** - * Retrieves matrix of features in flat format. - * - * @return matrix of features in flat format - */ - public double[] getFeatures() { - return storage.get(FEATURES_KEY); - } - - /** - * Sets number of rows. - * - * @param rows number of rows - */ - public void setRows(int rows) { - storage.put(ROWS_KEY, rows); - } - - /** - * Retrieves number of rows. - * - * @return number of rows - */ - public int getRows() { - return storage.get(ROWS_KEY); - } - - /** - * Removes all data associated with the partition. - */ - @Override public void close() { - storage.remove(FEATURES_KEY); - storage.remove(ROWS_KEY); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLearnLabeledDatasetPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLearnLabeledDatasetPartition.java deleted file mode 100644 index 7bac8cc036bb4..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/DLearnLabeledDatasetPartition.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * 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.ml.dlearn.dataset.part; - -import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; - -/** - * Interface which provides simple dataset API which allows to get or set an underlying feature matrix in flat format - * and vector of labels. - */ -public class DLearnLabeledDatasetPartition extends DLeanDatasetPartition { - /** Key of label values. */ - private static final String LABELS_KEY = "labels"; - - /** Learning partition storage. */ - private final DLearnPartitionStorage storage; - - /** - * Constructs a new instance of labeled dataset partition. - * - * @param storage learning partition storage. - */ - public DLearnLabeledDatasetPartition(DLearnPartitionStorage storage) { - super(storage); - this.storage = storage; - } - - /** - * Sets labels. - * - * @param labels labels - */ - public void setLabels(L[] labels) { - storage.put(LABELS_KEY, labels); - } - - /** - * Retrieves labels. - * - * @return labels - */ - public L[] getLabels() { - return storage.get(LABELS_KEY); - } - - /** - * Removes all data associated with the partition. - */ - @Override public void close() { - super.close(); - storage.remove(LABELS_KEY); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/package-info.java deleted file mode 100644 index f47f3c903cdf9..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/dataset/part/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Root package for different d-learn partitions. - */ -package org.apache.ignite.ml.dlearn.dataset.part; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/package-info.java deleted file mode 100644 index 51132cb7b3b36..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Root package for d-learn (distributed learning) framework. - */ -package org.apache.ignite.ml.dlearn; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextPartitionKey.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextPartitionKey.java deleted file mode 100644 index 45d20cb012f21..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/DLearnContextPartitionKey.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * 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.ml.dlearn.utils; - -import java.io.Serializable; -import java.util.Objects; -import java.util.UUID; - -/** - * Key used to identify d-learn partition objects. - */ -public class DLearnContextPartitionKey implements Serializable { - /** */ - private static final long serialVersionUID = 9005844909381326835L; - - /** Index of partition. */ - private final int part; - - /** Id of learning context. */ - private final UUID learningCtxId; - - /** Key of the object. */ - private final String key; - - /** - * Constructs a new instance of learning context partition key. - * - * @param part partition index - * @param learningCtxId learning context id - * @param key key - */ - public DLearnContextPartitionKey(int part, UUID learningCtxId, String key) { - this.part = part; - this.learningCtxId = learningCtxId; - this.key = key; - } - - /** */ - public int getPart() { - return part; - } - - /** */ - public UUID getLearningCtxId() { - return learningCtxId; - } - - /** */ - public String getKey() { - return key; - } - - /** */ - @Override public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - DLearnContextPartitionKey key1 = (DLearnContextPartitionKey)o; - return part == key1.part && - Objects.equals(learningCtxId, key1.learningCtxId) && - Objects.equals(key, key1.key); - } - - /** */ - @Override public int hashCode() { - return Objects.hash(part, learningCtxId, key); - } - - /** */ - @Override public String toString() { - return "DLearnContextPartitionKey{" + - "part=" + part + - ", learningCtxId=" + learningCtxId + - ", key='" + key + '\'' + - '}'; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/package-info.java deleted file mode 100644 index c09fbfea59f5e..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dlearn/utils/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Contains util classes used in d-learn module. - */ -package org.apache.ignite.ml.dlearn.utils; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java deleted file mode 100644 index e3496b6b6084e..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * 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.ml.regressions.linear; - -import org.apache.ignite.ml.Trainer; -import org.apache.ignite.ml.dlearn.DLearnContext; -import org.apache.ignite.ml.dlearn.dataset.part.DLearnLabeledDatasetPartition; - -/** */ -public class LinearRegressionLSQRTrainer implements Trainer> { - /** */ - private static final String A_NAME = "a"; - - /** */ - @Override public LinearRegressionModel train(DLearnContext learningCtx) { - preProcessContext(learningCtx); - -// DistributedLSQR lsqr = new DistributedLSQR<>( -// learningCtx, -// part -> part.get(A_NAME), -// LabeledDatasetLearningContextPartition::getLabels -// ); -// -// LSQRResult res = lsqr.solve(0, 1e-8, 1e-8, 1e8, -1, false, null); -// -// postProcessContext(learningCtx); -// -// double[] x = res.getX(); -// double[] weights = Arrays.copyOfRange(x, 1, x.length); -// double intercept = x[0]; -// -// return new LinearRegressionModel(new DenseLocalOnHeapVector(weights), intercept); - return null; - } - - /** - * Processing of given learning context before training. - */ - private void preProcessContext(DLearnContext learningCtx) { - learningCtx.compute(part -> { - double[] features = part.getFeatures(); - - int rows = part.getRows(); - double[] a = addInterceptCoefficientColumn(features, rows); - -// part.put(A_NAME, a); - }); - } - - /** - * Processing of given learning context after training. - */ - private void postProcessContext(DLearnContext learningCtx) { -// learningCtx.compute(part -> part.remove(A_NAME)); - } - - /** - * Adds intercept coefficient (1.0) columns to the matrix. - * - * @return matrix with intercept coefficient column - */ - private double[] addInterceptCoefficientColumn(double[] features, int rows) { - double[] res = new double[features.length + rows]; - System.arraycopy(features, 0, res, rows, features.length); - for (int i = 0; i < rows; i++) - res[i] = 1.0; - return res; - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/api/DatasetWrapperTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/api/DatasetWrapperTest.java new file mode 100644 index 0000000000000..9efd630797a93 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/api/DatasetWrapperTest.java @@ -0,0 +1,4 @@ +package org.apache.ignite.ml.dataset.api; + +public class DatasetWrapperTest { +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilderTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilderTest.java new file mode 100644 index 0000000000000..75accf62ce9fa --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilderTest.java @@ -0,0 +1,95 @@ +package org.apache.ignite.ml.dataset.impl.cache; + +import java.util.Collection; +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests for {@link CacheBasedDatasetBuilder}. + */ +public class CacheBasedDatasetBuilderTest extends GridCommonAbstractTest { + /** Number of nodes in grid */ + private static final int NODE_COUNT = 10; + + /** Ignite instance. */ + private Ignite ignite; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + for (int i = 1; i <= NODE_COUNT; i++) + startGrid(i); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() { + stopAllGrids(); + } + + /** + * {@inheritDoc} + */ + @Override protected void beforeTest() throws Exception { + /* Grid instance. */ + ignite = grid(NODE_COUNT); + ignite.configuration().setPeerClassLoadingEnabled(true); + IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); + } + + /** + * Tests that partitions of the dataset cache are placed on the same nodes as upstream cache. + */ + public void testBuild() { + IgniteCache upstreamCache = createTestCache(100, 10); + CacheBasedDatasetBuilder builder = + new CacheBasedDatasetBuilder<>( + ignite, + upstreamCache, + (upstream, upstreamSize) -> upstreamSize, + (upstream, upstreamSize, ctx) -> null + ); + + CacheBasedDataset dataset = builder.build(); + + Affinity upstreamAffinity = ignite.affinity(upstreamCache.getName()); + Affinity datasetAffinity = ignite.affinity(dataset.getDatasetCache().getName()); + + int upstreamPartitions = upstreamAffinity.partitions(); + int datasetPartitions = datasetAffinity.partitions(); + + assertEquals(upstreamPartitions, datasetPartitions); + + for (int part = 0; part < upstreamPartitions; part++) { + Collection upstreamPartNodes = upstreamAffinity.mapPartitionToPrimaryAndBackups(part); + Collection datasetPartNodes = datasetAffinity.mapPartitionToPrimaryAndBackups(part); + + assertEqualsCollections(upstreamPartNodes, datasetPartNodes); + } + } + + /** + * Generate an Ignite Cache with the specified size and number of partitions for testing purposes. + * + * @param size size of an Ignite Cache + * @param parts number of partitions + * @return Ignite Cache instance + */ + private IgniteCache createTestCache(int size, int parts) { + CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); + cacheConfiguration.setName(UUID.randomUUID().toString()); + cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, parts)); + + IgniteCache cache = ignite.createCache(cacheConfiguration); + + for (int i = 0; i < size; i++) + cache.put(i, "DATA_" + i); + + return cache; + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java similarity index 52% rename from modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java rename to modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java index 698f3e26f5d52..4e5f7475a8b05 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java @@ -1,21 +1,4 @@ -/* - * 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.ml.dlearn.context.cache; +package org.apache.ignite.ml.dataset.impl.cache; import java.util.ArrayList; import java.util.HashMap; @@ -38,15 +21,9 @@ import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.lang.IgnitePredicate; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformers; -import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; -import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -/** - * Tests for {@link CacheDLearnContext}. - */ -public class CacheDLearnContextTest extends GridCommonAbstractTest { +public class CacheBasedDatasetTest extends GridCommonAbstractTest { /** Number of nodes in grid */ private static final int NODE_COUNT = 4; @@ -74,77 +51,6 @@ public class CacheDLearnContextTest extends GridCommonAbstractTest { IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); } - /** */ - public void testClose() { - IgniteCache data = generateTestData(2, 0); - - CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); - CacheDLearnContext> ctx = ctxFactory.createContext(); - IgniteCache learningCtxCache = ignite.cache(ctx.getLearningCtxCacheName()); - - // context cache contains 2 partitions, each partition contains cache name and partition number - assertEquals(4, learningCtxCache.size()); - - ctx.close(); - - // all data were removed from context cache - assertEquals(0, learningCtxCache.size()); - } - - /** */ - public void testCloseDerivativeContext() { - IgniteCache data = generateTestData(2, 0); - - CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); - CacheDLearnContext> ctx = ctxFactory.createContext(); - IgniteCache learningCtxCache = ignite.cache(ctx.getLearningCtxCacheName()); - - // context cache contains 2 partitions, each partition contains cache name and partition number - assertEquals(4, learningCtxCache.size()); - - DLearnDataset dataset = ctx.transform(DLearnContextTransformers.cacheToDataset((k, v) -> new double[0])); - - // features and rows were added into both partitions - assertEquals(8, learningCtxCache.size()); - - dataset.close(); - - // features and rows were removed - assertEquals(4, learningCtxCache.size()); - - ctx.close(); - - // all data were removed from context cache - assertEquals(0, learningCtxCache.size()); - } - - /** */ - public void testCloseBaseContext() { - IgniteCache data = generateTestData(2, 0); - - CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); - CacheDLearnContext> ctx = ctxFactory.createContext(); - IgniteCache learningCtxCache = ignite.cache(ctx.getLearningCtxCacheName()); - - // context cache contains 2 partitions, each partition contains cache name and partition number - assertEquals(4, learningCtxCache.size()); - - DLearnDataset dataset = ctx.transform(DLearnContextTransformers.cacheToDataset((k, v) -> new double[0])); - - // features and rows were added into both partitions - assertEquals(8, learningCtxCache.size()); - - ctx.close(); - - // 2 partitions with initial data were removed - assertEquals(4, learningCtxCache.size()); - - dataset.close(); - - // all data were removed from context cache - assertEquals(0, learningCtxCache.size()); - } - /** * Tests that partitions of upstream cache and context cache are reserved during computations on base learning * context. Reservation means that partitions won't be unloaded from the node before computation will be completed. @@ -152,28 +58,36 @@ public void testCloseBaseContext() { public void testPartitionExchangeDuringComputeWithReturnCallOnBaseContext() { int partitions = 4; - IgniteCache data = generateTestData(4, 0); + IgniteCache upstreamCache = generateTestData(4, 0); + + CacheBasedDatasetBuilder builder = + new CacheBasedDatasetBuilder<>( + ignite, + upstreamCache, + (upstream, upstreamSize) -> upstreamSize, + (upstream, upstreamSize, ctx) -> null + ); - // create learning context - CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); - CacheDLearnContext> ctx = ctxFactory.createContext(); + CacheBasedDataset dataset = builder.build(); assertTrue("Before computation all partitions should not be reserved", - areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); + areAllPartitionsNotReserved(upstreamCache.getName(), dataset.getDatasetCache().getName())); - IgniteAtomicLong numOfStartedComputations = ignite.atomicLong(UUID.randomUUID().toString(), 0, true); + UUID numOfStartedComputationsId = UUID.randomUUID(); + IgniteAtomicLong numOfStartedComputations = ignite.atomicLong(numOfStartedComputationsId.toString(), 0, true); - IgniteLock computationsLock = ignite.reentrantLock(UUID.randomUUID().toString(), false, true, true); + UUID computationsLockId = UUID.randomUUID(); + IgniteLock computationsLock = ignite.reentrantLock(computationsLockId.toString(), false, true, true); // lock computations lock to stop computations in the middle computationsLock.lock(); try { - new Thread(() -> ctx.compute((part, partIndex) -> { + new Thread(() -> dataset.compute((part, partIndex) -> { // track number of started computations - numOfStartedComputations.incrementAndGet(); - computationsLock.lock(); - computationsLock.unlock(); + ignite.atomicLong(numOfStartedComputationsId.toString(), 0, false).incrementAndGet(); + ignite.reentrantLock(computationsLockId.toString(), false, true, false).lock(); + ignite.reentrantLock(computationsLockId.toString(), false, true, false).unlock(); return 42; }, (a, b) -> 42)).start(); // wait all computations to start @@ -182,14 +96,14 @@ public void testPartitionExchangeDuringComputeWithReturnCallOnBaseContext() { } assertTrue("During computation all partitions should be reserved", - areAllPartitionsReserved(data.getName(), ctx.getLearningCtxCacheName())); + areAllPartitionsReserved(upstreamCache.getName(), dataset.getDatasetCache().getName())); } finally { computationsLock.unlock(); } assertTrue("All partitions should be released", - areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); + areAllPartitionsNotReserved(upstreamCache.getName(), dataset.getDatasetCache().getName())); } /** @@ -199,67 +113,27 @@ public void testPartitionExchangeDuringComputeWithReturnCallOnBaseContext() { public void testPartitionExchangeDuringComputeCallOnBaseContext() { int partitions = 4; - IgniteCache data = generateTestData(4, 0); - - // create learning context - CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); - CacheDLearnContext> ctx = ctxFactory.createContext(); - - assertTrue("Before computation all partitions should not be reserved", - areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); - - IgniteAtomicLong numOfStartedComputations = ignite.atomicLong(UUID.randomUUID().toString(), 0, true); - - IgniteLock computationsLock = ignite.reentrantLock(UUID.randomUUID().toString(), false, true, true); + IgniteCache upstreamCache = generateTestData(4, 0); - // lock computations lock to stop computations in the middle - computationsLock.lock(); + CacheBasedDatasetBuilder builder = + new CacheBasedDatasetBuilder<>( + ignite, + upstreamCache, + (upstream, upstreamSize) -> upstreamSize, + (upstream, upstreamSize, ctx) -> null + ); - try { - new Thread(() -> ctx.compute((part, partIndex) -> { - // track number of started computations - numOfStartedComputations.incrementAndGet(); - computationsLock.lock(); - computationsLock.unlock(); - })).start(); - // wait all computations to start + CacheBasedDataset dataset = builder.build(); - while (numOfStartedComputations.get() < partitions) { - } - - assertTrue("During computation all partitions should be reserved", - areAllPartitionsReserved(data.getName(), ctx.getLearningCtxCacheName())); - } - finally { - computationsLock.unlock(); - } - - assertTrue("All partitions should be released", - areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); - } - - /** - * Tests that only partitions of context cache are reserved during computations on base learning - * context. Reservation means that partitions won't be unloaded from the node before computation will be completed. - * Partitions of the upstream cache should not be reserved. - */ - public void testPartitionExchangeDuringComputeWithReturnCallOnDerivativeContext() throws InterruptedException { - int partitions = 4; - - IgniteCache data = generateTestData(4, 0); - - // create learning context - CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); - CacheDLearnContext> ctx = ctxFactory.createContext(); - - DLearnDataset dataset = ctx.transform(DLearnContextTransformers.cacheToDataset((k, v) -> new double[10])); assertTrue("Before computation all partitions should not be reserved", - areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); + areAllPartitionsNotReserved(upstreamCache.getName(), dataset.getDatasetCache().getName())); - IgniteAtomicLong numOfStartedComputations = ignite.atomicLong(UUID.randomUUID().toString(), 0, true); + UUID numOfStartedComputationsId = UUID.randomUUID(); + IgniteAtomicLong numOfStartedComputations = ignite.atomicLong(numOfStartedComputationsId.toString(), 0, true); - IgniteLock computationsLock = ignite.reentrantLock(UUID.randomUUID().toString(), false, true, true); + UUID computationsLockId = UUID.randomUUID(); + IgniteLock computationsLock = ignite.reentrantLock(computationsLockId.toString(), false, true, true); // lock computations lock to stop computations in the middle computationsLock.lock(); @@ -267,78 +141,24 @@ public void testPartitionExchangeDuringComputeWithReturnCallOnDerivativeContext( try { new Thread(() -> dataset.compute((part, partIndex) -> { // track number of started computations - numOfStartedComputations.incrementAndGet(); - computationsLock.lock(); - computationsLock.unlock(); - return 42; - }, (a, b) -> 42)).start(); - // wait all computations to start - - while (numOfStartedComputations.get() < partitions) { - } - - assertTrue("During computation all partitions of the context cache should be reserved", - areAllPartitionsReserved(ctx.getLearningCtxCacheName())); - assertTrue("During computation all partitions of the data cache should be reserved", - areAllPartitionsNotReserved(data.getName())); - } - finally { - computationsLock.unlock(); - } - - assertTrue("All partitions should be released", - areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); - } - - /** - * Tests that only partitions of context cache are reserved during computations on base learning - * context. Reservation means that partitions won't be unloaded from the node before computation will be completed. - * Partitions of the upstream cache should not be reserved. - */ - public void testPartitionExchangeDuringComputeCallOnDerivativeContext() throws InterruptedException { - int partitions = 4; - - IgniteCache data = generateTestData(4, 0); - - // create learning context - CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); - CacheDLearnContext> ctx = ctxFactory.createContext(); - - DLearnDataset dataset = ctx.transform(DLearnContextTransformers.cacheToDataset((k, v) -> new double[10])); - - assertTrue("Before computation all partitions should not be reserved", - areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); - - IgniteAtomicLong numOfStartedComputations = ignite.atomicLong(UUID.randomUUID().toString(), 0, true); - - IgniteLock computationsLock = ignite.reentrantLock(UUID.randomUUID().toString(), false, true, true); - - // lock computations lock to stop computations in the middle - computationsLock.lock(); - - try { - new Thread(() -> dataset.compute((part, partIndex) -> { - // track number of started computations - numOfStartedComputations.incrementAndGet(); - computationsLock.lock(); - computationsLock.unlock(); + ignite.atomicLong(numOfStartedComputationsId.toString(), 0, false).incrementAndGet(); + ignite.reentrantLock(computationsLockId.toString(), false, true, false).lock(); + ignite.reentrantLock(computationsLockId.toString(), false, true, false).unlock(); })).start(); // wait all computations to start while (numOfStartedComputations.get() < partitions) { } - assertTrue("During computation all partitions of the context cache should be reserved", - areAllPartitionsReserved(ctx.getLearningCtxCacheName())); - assertTrue("During computation all partitions of the data cache should be reserved", - areAllPartitionsNotReserved(data.getName())); + assertTrue("During computation all partitions should be reserved", + areAllPartitionsReserved(upstreamCache.getName(), dataset.getDatasetCache().getName())); } finally { computationsLock.unlock(); } assertTrue("All partitions should be released", - areAllPartitionsNotReserved(data.getName(), ctx.getLearningCtxCacheName())); + areAllPartitionsNotReserved(upstreamCache.getName(), dataset.getDatasetCache().getName())); } /** diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilderTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilderTest.java new file mode 100644 index 0000000000000..193000380b6f9 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilderTest.java @@ -0,0 +1,4 @@ +package org.apache.ignite.ml.dataset.impl.local; + +public class LocalDatasetBuilderTest { +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetTest.java new file mode 100644 index 0000000000000..b7b873ed5e9b0 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetTest.java @@ -0,0 +1,4 @@ +package org.apache.ignite.ml.dataset.impl.local; + +public class LocalDatasetTest { +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactoryTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactoryTest.java deleted file mode 100644 index 03edc6eb4c752..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/CacheDLearnContextFactoryTest.java +++ /dev/null @@ -1,118 +0,0 @@ -/* - * 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.ml.dlearn.context.cache; - -import java.util.UUID; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -/** - * Tests for {@link CacheDLearnContextFactory}. - */ -public class CacheDLearnContextFactoryTest extends GridCommonAbstractTest { - /** Number of nodes in grid */ - private static final int NODE_COUNT = 10; - - /** */ - private Ignite ignite; - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - for (int i = 1; i <= NODE_COUNT; i++) - startGrid(i); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() { - stopAllGrids(); - } - - /** - * {@inheritDoc} - */ - @Override protected void beforeTest() throws Exception { - /* Grid instance. */ - ignite = grid(NODE_COUNT); - ignite.configuration().setPeerClassLoadingEnabled(true); - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - } - - /** - * Test that partitions of the context created on top of the dataset are placed on the same nodes as initial - * dataset (number of partitions are less than number of nodes). - */ - public void testAffinityWithNumberOfPartitionsLessThanNodes() { - testAffinity(generateTestData(5, 10), 5); - } - - /** - * Test that partitions of the context created on top of the dataset are placed on the same nodes as initial - * dataset (number of partitions are greater than number of nodes). - */ - public void testAffinityWithNumberOfPartitionsGreaterThanNodes() { - testAffinity(generateTestData(50, 10), 50); - } - - /** */ - private void testAffinity(IgniteCache data, int partitions) { - CacheDLearnContextFactory ctxFactory = new CacheDLearnContextFactory<>(ignite, data); - - CacheDLearnContext> ctx = ctxFactory.createContext(); - - Affinity dataAffinity = ignite.affinity(data.getName()); - Affinity ctxAffinity = ignite.affinity(ctx.getLearningCtxCacheName()); - - assertEquals(partitions, dataAffinity.partitions()); - assertEquals(partitions, ctxAffinity.partitions()); - - for (int part = 0; part < partitions; part++) { - ClusterNode dataPartPrimaryNode = dataAffinity.mapPartitionToNode(part); - ClusterNode ctxPartPrimaryNode = ctxAffinity.mapPartitionToNode(part); - - assertNotNull(dataPartPrimaryNode); - assertNotNull(ctxPartPrimaryNode); - assertEquals(dataPartPrimaryNode, ctxPartPrimaryNode); - } - } - - /** - * Generates Ignite Cache with data for tests. - * - * @return Ignite Cache with data for tests - */ - private IgniteCache generateTestData(int partitions, int backups) { - CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); - - cacheConfiguration.setName(UUID.randomUUID().toString()); - cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, partitions)); - cacheConfiguration.setBackups(backups); - - IgniteCache cache = ignite.createCache(cacheConfiguration); - - for (int i = 0; i < partitions * 10; i++) - cache.put(i, String.valueOf(i)); - - return cache; - } -} \ No newline at end of file diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunctionTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunctionTest.java deleted file mode 100644 index 840ecc1323f07..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/cache/utils/DLearnPartitionAffinityFunctionTest.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * 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.ml.dlearn.context.cache.utils; - -/** - * Tests for {@link DLearnPartitionAffinityFunction}. - */ -public class DLearnPartitionAffinityFunctionTest { -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextTest.java deleted file mode 100644 index dd4bc3b29bd90..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dlearn/context/local/LocalDLearnContextTest.java +++ /dev/null @@ -1,125 +0,0 @@ -/* - * 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.ml.dlearn.context.local; - -import java.util.HashMap; -import java.util.Map; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformers; -import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; -import org.apache.ignite.ml.dlearn.utils.DLearnContextPartitionKey; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; - -/** - * Tests for {@link LocalDLearnContext}. - */ -public class LocalDLearnContextTest { - /** */ - @Test - public void testClose() { - Map data = generateTestData(); - - LocalDLearnContextFactory ctxFactory = new LocalDLearnContextFactory<>(data, 2); - - LocalDLearnContext> ctx = ctxFactory.createContext(); - Map learningCtxMap = ctx.getLearningCtxMap(); - - // context cache contains 2 partitions with initial data - assertEquals(2, learningCtxMap.size()); - - ctx.close(); - - // all data were removed from context cache - assertEquals(0, learningCtxMap.size()); - } - - /** */ - @Test - public void testCloseDerivativeContext() { - Map data = generateTestData(); - - LocalDLearnContextFactory ctxFactory = new LocalDLearnContextFactory<>(data, 2); - - LocalDLearnContext> ctx = ctxFactory.createContext(); - Map learningCtxMap = ctx.getLearningCtxMap(); - - // context cache contains 2 partitions with initial data - assertEquals(2, learningCtxMap.size()); - - DLearnDataset dataset = ctx.transform(DLearnContextTransformers.localToDataset((k, v) -> new double[0])); - - // features and rows were added into both partitions - assertEquals(6, learningCtxMap.size()); - - dataset.close(); - - // features and rows were removed - assertEquals(2, learningCtxMap.size()); - - ctx.close(); - - // all data were removed from context cache - assertEquals(0, learningCtxMap.size()); - } - - /** */ - @Test - public void testCloseBaseContext() { - Map data = generateTestData(); - - LocalDLearnContextFactory ctxFactory = new LocalDLearnContextFactory<>(data, 2); - - LocalDLearnContext> ctx = ctxFactory.createContext(); - Map learningCtxMap = ctx.getLearningCtxMap(); - - // context cache contains 2 partitions with initial data - assertEquals(2, learningCtxMap.size()); - - DLearnDataset dataset = ctx.transform(DLearnContextTransformers.localToDataset((k, v) -> new double[0])); - - // features and rows were added into both partitions - assertEquals(6, learningCtxMap.size()); - - ctx.close(); - - // 2 partitions with initial data were removed - assertEquals(4, learningCtxMap.size()); - - dataset.close(); - - // all data were removed from context cache - assertEquals(0, learningCtxMap.size()); - } - - /** - * Generates list with data for test. - * - * @return list with data for test. - */ - private Map generateTestData() { - Map data = new HashMap<>(); - - data.put(1, "TEST1"); - data.put(2, "TEST2"); - data.put(3, "TEST3"); - data.put(4, "TEST4"); - - return data; - } -} From c39b3ebf2d0793a1044c75663fa9cbbcb0777ae7 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Sat, 27 Jan 2018 18:17:34 +0300 Subject: [PATCH 20/38] IGNITE-7437 Remove d-learn part from examples. --- .../ml/dlearn/CacheDatasetExample.java | 124 ---------- .../ml/dlearn/LocalDatasetExample.java | 122 --------- .../ml/dlearn/TransformerExample.java | 231 ------------------ .../examples/ml/dlearn/package-info.java | 22 -- 4 files changed, 499 deletions(-) delete mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlearn/CacheDatasetExample.java delete mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlearn/LocalDatasetExample.java delete mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlearn/TransformerExample.java delete mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dlearn/package-info.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/CacheDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/CacheDatasetExample.java deleted file mode 100644 index 056d6c2d25546..0000000000000 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/CacheDatasetExample.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * 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.examples.ml.dlearn; - -import java.util.Arrays; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.Ignition; -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContext; -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContextFactory; -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformers; -import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; - -/** - * How to create a d-learn dataset from an existing Ignite Cache? - */ -public class CacheDatasetExample { - /** Run example. */ - public static void main(String[] args) { - try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { - System.out.println(">>> D-Learn Cache Dataset example started."); - - IgniteCache persons = createCache(ignite); - - // Initialization of the d-learn context. After this step context cache will be created with partitions - // placed on the same nodes as the upstream Ignite Cache. - CacheDLearnContext> ctx = - new CacheDLearnContextFactory<>(ignite, persons).createContext(); - - // Loading of the d-learn context. During this step data will be transferred from the upstream cache to - // context cache with specified transformation (it will be performed locally because partitions are on the - // same nodes). In this case for every partition in upstream cache will be created labeled dataset partition - // and this new partition will be filled with help of specified feature and label extractors. - DLearnDataset dataset = ctx.transform(DLearnContextTransformers.cacheToDataset((k, v) -> { - double[] row = new double[2]; - row[0] = v.age; - row[1] = v.salary; - return row; - })); - - // Calculation of the mean value. This calculation will be performed in map-reduce manner. - double[] mean = dataset.mean(new int[] {0, 1}); - System.out.println("Mean \n\t" + Arrays.toString(mean)); - - // Calculation of the standard deviation. This calculation will be performed in map-reduce manner. - double[] std = dataset.std(new int[] {0, 1}); - System.out.println("Standard deviation \n\t" + Arrays.toString(std)); - - // Calculation of the covariance matrix. This calculation will be performed in map-reduce manner. - double[][] cov = dataset.cov(new int[] {0, 1}); - System.out.println("Covariance matrix "); - for (double[] row : cov) - System.out.println("\t" + Arrays.toString(row)); - - // Calculation of the correlation matrix. This calculation will be performed in map-reduce manner. - double[][] corr = dataset.corr(new int[] {0, 1}); - System.out.println("Correlation matrix "); - for (double[] row : corr) - System.out.println("\t" + Arrays.toString(row)); - - System.out.println(">>> D-Learn Cache Dataset example completed."); - } - } - - /** */ - private static IgniteCache createCache(Ignite ignite) { - IgniteCache persons = ignite.createCache("PERSONS"); - persons.put(1, new Person("Mike", 42, 10000)); - persons.put(2, new Person("John", 32, 64000)); - persons.put(3, new Person("George", 53, 120000)); - persons.put(4, new Person("Karl", 24, 70000)); - return persons; - } - - /** */ - private static class Person { - /** */ - private final String name; - - /** */ - private final double age; - - /** */ - private final double salary; - - /** */ - public Person(String name, double age, double salary) { - this.name = name; - this.age = age; - this.salary = salary; - } - - /** */ - public String getName() { - return name; - } - - /** */ - public double getAge() { - return age; - } - - /** */ - public double getSalary() { - return salary; - } - } -} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/LocalDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/LocalDatasetExample.java deleted file mode 100644 index 1e91ff0b620cb..0000000000000 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/LocalDatasetExample.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * 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.examples.ml.dlearn; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import org.apache.ignite.Ignite; -import org.apache.ignite.Ignition; -import org.apache.ignite.ml.dlearn.context.local.LocalDLearnContext; -import org.apache.ignite.ml.dlearn.context.local.LocalDLearnContextFactory; -import org.apache.ignite.ml.dlearn.context.local.LocalDLearnPartition; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformers; -import org.apache.ignite.ml.dlearn.dataset.DLearnDataset; - -/** - * How to create a d-learn dataset from an existing local data? - */ -public class LocalDatasetExample { - /** Run example. */ - public static void main(String[] args) { - try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { - System.out.println(">>> D-Learn Local Dataset example started."); - - Map persons = createCache(ignite); - - // Initialization of the d-learn context. This context uses local hash map as an underlying storage. - LocalDLearnContext> ctx = - new LocalDLearnContextFactory<>(persons, 2).createContext(); - - // Loading of the d-learn context. During this step data will be transferred from the generic local - // partition to dataset partition. - DLearnDataset dataset = ctx.transform(DLearnContextTransformers.localToDataset((k, v) -> { - double[] row = new double[2]; - row[0] = v.age; - row[1] = v.salary; - return row; - })); - - // Calculation of the mean value. This calculation will be performed in map-reduce manner. - double[] mean = dataset.mean(new int[] {0, 1}); - System.out.println("Mean \n\t" + Arrays.toString(mean)); - - // Calculation of the standard deviation. This calculation will be performed in map-reduce manner. - double[] std = dataset.std(new int[] {0, 1}); - System.out.println("Standard deviation \n\t" + Arrays.toString(std)); - - // Calculation of the covariance matrix. This calculation will be performed in map-reduce manner. - double[][] cov = dataset.cov(new int[] {0, 1}); - System.out.println("Covariance matrix "); - for (double[] row : cov) - System.out.println("\t" + Arrays.toString(row)); - - // Calculation of the correlation matrix. This calculation will be performed in map-reduce manner. - double[][] corr = dataset.corr(new int[] {0, 1}); - System.out.println("Correlation matrix "); - for (double[] row : corr) - System.out.println("\t" + Arrays.toString(row)); - - System.out.println(">>> D-Learn Local Dataset example completed."); - } - } - - /** */ - private static Map createCache(Ignite ignite) { - Map persons = new HashMap<>(); - persons.put(1, new Person("Mike", 42, 10000)); - persons.put(2, new Person("John", 32, 64000)); - persons.put(3, new Person("George", 53, 120000)); - persons.put(4, new Person("Karl", 24, 70000)); - return persons; - } - - /** */ - private static class Person { - /** */ - private final String name; - - /** */ - private final double age; - - /** */ - private final double salary; - - /** */ - public Person(String name, double age, double salary) { - this.name = name; - this.age = age; - this.salary = salary; - } - - /** */ - public String getName() { - return name; - } - - /** */ - public double getAge() { - return age; - } - - /** */ - public double getSalary() { - return salary; - } - } -} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/TransformerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/TransformerExample.java deleted file mode 100644 index 13e94043b9efa..0000000000000 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/TransformerExample.java +++ /dev/null @@ -1,231 +0,0 @@ -/* - * 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.examples.ml.dlearn; - -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.Ignition; -import org.apache.ignite.ml.dlearn.DLearnContext; -import org.apache.ignite.ml.dlearn.DLearnPartitionStorage; -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContext; -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnContextFactory; -import org.apache.ignite.ml.dlearn.context.cache.CacheDLearnPartition; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformer; -import org.apache.ignite.ml.dlearn.context.transformer.DLearnContextTransformers; -import org.apache.ignite.ml.dlearn.dataset.AbstractDLearnContextWrapper; -import org.apache.ignite.ml.dlearn.dataset.DLearnLabeledDataset; -import org.apache.ignite.ml.dlearn.dataset.part.DLearnLabeledDatasetPartition; - -/** - * How to transform dataset into algorithm-specific view? - */ -public class TransformerExample { - /** Run example. */ - public static void main(String[] args) { - try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { - System.out.println(">>> D-Learn Dataset example started."); - - IgniteCache persons = createCache(ignite); - - // Initialization of the d-learn context. After this step context cache will be created with partitions - // placed on the same nodes as the upstream Ignite Cache. - CacheDLearnContext> ctx = - new CacheDLearnContextFactory<>(ignite, persons).createContext(); - - // Loading of the d-learn context. During this step data will be transferred from the upstream cache to - // context cache with specified transformation (it will be performed locally because partitions are on the - // same nodes). In this case for every partition in upstream cache will be created labeled dataset partition - // and this new partition will be filled with help of specified feature and label extractors. - DLearnLabeledDataset dataset = ctx.transform(DLearnContextTransformers.cacheToLabeledDataset( - (k, v) -> new double[]{ v.age }, - (k, v) -> v.salary - )); - - AlgorithmSpecificContext algorithmSpecificCtx = dataset.transform(new AlgorithmSpecificTransformer()); - System.out.println("Result: " + algorithmSpecificCtx.solve()); - } - } - - /** - * Algorithm-specific partition. - */ - private static class AlgorithmSpecificPartition implements AutoCloseable { - /** */ - private static final String MATRIX_A_KEY = "matrix_a"; - - /** */ - private static final String VECTOR_B_KEY = "vector_b"; - - /** */ - private static final String ROWS_KEY = "rows"; - - /** */ - private final DLearnPartitionStorage storage; - - /** */ - public AlgorithmSpecificPartition(DLearnPartitionStorage storage) { - this.storage = storage; - } - - /** */ - public double[] getMatrixA() { - return storage.get(MATRIX_A_KEY); - } - - /** */ - public void setMatrixA(double[] matrixA) { - storage.put(MATRIX_A_KEY, matrixA); - } - - /** */ - public Integer getRows() { - return storage.get(ROWS_KEY); - } - - /** */ - public void setRows(int rows) { - storage.put(ROWS_KEY, rows); - } - - /** */ - public double[] getVectorB() { - return storage.get(VECTOR_B_KEY); - } - - /** */ - public void setVectorB(double[] vectorB) { - storage.put(VECTOR_B_KEY, vectorB); - } - - /** */ - @Override public void close() { - storage.remove(MATRIX_A_KEY); - storage.remove(VECTOR_B_KEY); - storage.remove(ROWS_KEY); - } - } - - /** - * Algorithm-specific representation of the context. - */ - private static class AlgorithmSpecificContext extends AbstractDLearnContextWrapper { - /** */ - public AlgorithmSpecificContext( - DLearnContext delegate) { - super(delegate); - } - - /** */ - public double solve() { - return compute(part -> { - double[] matrixA = part.getMatrixA(); - double[] vectorB = part.getVectorB(); - Integer rows = part.getRows(); - // do something to solve... - return 42.0; - }, (a, b) -> a == null ? b : a + b); - } - } - - /** - * Algorithm-specific transformer. - */ - private static class AlgorithmSpecificTransformer implements DLearnContextTransformer, - AlgorithmSpecificPartition, AlgorithmSpecificContext> { - /** */ - private static final long serialVersionUID = 2109144841306143061L; - - /** */ - @Override public void transform(DLearnLabeledDatasetPartition oldPart, - AlgorithmSpecificPartition newPart) { - double[] features = oldPart.getFeatures(); - - if (features != null) { - Object[] labels = oldPart.getLabels(); - int rows = oldPart.getRows(); - - double[] aMatrix = new double[features.length + rows]; - double[] bVector = new double[rows]; - - for (int i = 0; i < rows; i++) { - aMatrix[i] = 1.0; - bVector[i] = (Double) labels[i]; - } - - newPart.setMatrixA(aMatrix); - newPart.setVectorB(bVector); - newPart.setRows(rows); - } - } - - /** */ - @Override public AlgorithmSpecificContext wrapContext( - DLearnContext ctx) { - return new AlgorithmSpecificContext(ctx); - } - - /** */ - @Override public AlgorithmSpecificPartition createPartition(DLearnPartitionStorage storage) { - return new AlgorithmSpecificPartition(storage); - } - } - - /** */ - private static IgniteCache createCache(Ignite ignite) { - IgniteCache persons = ignite.createCache("PERSONS"); - persons.put(1, new Person("Mike", 42, 10000)); - persons.put(2, new Person("John", 32, 64000)); - persons.put(3, new Person("George", 53, 120000)); - persons.put(4, new Person("Karl", 24, 70000)); - return persons; - } - - /** */ - private static class Person { - /** */ - private final String name; - - /** */ - private final double age; - - /** */ - private final double salary; - - /** */ - public Person(String name, double age, double salary) { - this.name = name; - this.age = age; - this.salary = salary; - } - - /** */ - public String getName() { - return name; - } - - /** */ - public double getAge() { - return age; - } - - /** */ - public double getSalary() { - return salary; - } - } -} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/package-info.java b/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/package-info.java deleted file mode 100644 index e3c92ec0a1b7f..0000000000000 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dlearn/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Machine learning d-learn framework examples. - */ -package org.apache.ignite.examples.ml.dlearn; \ No newline at end of file From a00ff930217c5d99901dc7491d5148b89f9c8788 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Mon, 29 Jan 2018 14:09:39 +0300 Subject: [PATCH 21/38] IGNITE-7437 Update examples for partition based dataset. --- .../AlgorithmSpecificDatasetExample.java | 171 ++++++++++++++++++ .../ml/dataset/CacheBasedDatasetExample.java | 32 +--- ...tExample.java => LocalDatasetExample.java} | 35 +--- .../examples/ml/dataset/model/Person.java | 58 ++++++ .../ignite/ml/dataset/api/SimpleDataset.java | 42 ++++- .../dataset/impl/cache/CacheBasedDataset.java | 7 +- .../dataset/impl/cache/util/ComputeUtils.java | 21 ++- 7 files changed, 296 insertions(+), 70 deletions(-) create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dataset/AlgorithmSpecificDatasetExample.java rename examples/src/main/java/org/apache/ignite/examples/ml/dataset/{LocalBasedDatasetExample.java => LocalDatasetExample.java} (72%) create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dataset/model/Person.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/AlgorithmSpecificDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/AlgorithmSpecificDatasetExample.java new file mode 100644 index 0000000000000..61a531e8ab350 --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/AlgorithmSpecificDatasetExample.java @@ -0,0 +1,171 @@ +/* + * 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.examples.ml.dataset; + +import com.github.fommil.netlib.BLAS; +import java.io.Serializable; +import java.util.Arrays; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.examples.ml.dataset.model.Person; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.DatasetFactory; +import org.apache.ignite.ml.dataset.api.DatasetWrapper; +import org.apache.ignite.ml.dataset.api.builder.data.SimpleLabeledDatasetDataBuilder; +import org.apache.ignite.ml.dataset.api.data.SimpleLabeledDatasetData; + +/** + * How to create a algorithm specific cache based dataset from an existing Ignite Cache? + */ +public class AlgorithmSpecificDatasetExample { + /** Run example. */ + public static void main(String[] args) { + try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { + System.out.println(">>> D-Learn Cache Dataset example started."); + + IgniteCache persons = createCache(ignite); + + // Creates a algorithm specific dataset to perform linear regression. + AlgorithmSpecificDataset dataset = DatasetFactory.create( + ignite, + persons, + (upstream, upstreamSize) -> new AlgorithmSpecificPartitionContext(), + new SimpleLabeledDatasetDataBuilder( + (k, v) -> new double[] {v.getAge()}, + (k, v) -> v.getSalary(), + 1 + ).andThen((data, ctx) -> { + double[] features = data.getFeatures(); + int rows = data.getRows(); + + // Makes a copy of features to supplement it by columns with values equal to 1.0. + double[] a = new double[features.length + rows]; + + for (int i = 0; i < rows; i++) + a[i] = 1.0; + + System.arraycopy(features, 0, a, rows, features.length); + + return new SimpleLabeledDatasetData(a, rows, data.getCols() + 1, data.getLabels()); + }) + ).wrap(AlgorithmSpecificDataset::new); + + // Trains linear regression model using gradient descent. + double[] linearRegressionMdl = new double[2]; + + for (int i = 0; i < 1000; i++) { + double[] gradient = dataset.gradient(linearRegressionMdl); + + if (BLAS.getInstance().dnrm2(gradient.length, gradient, 1) < 1e-4) + break; + + for (int j = 0; j < gradient.length; j++) + linearRegressionMdl[j] -= 0.1 / persons.size() * gradient[j]; + } + + System.out.println("Linear Regression Model: " + Arrays.toString(linearRegressionMdl)); + } + } + + /** Algorithm specific dataset. */ + private static class AlgorithmSpecificDataset + extends DatasetWrapper { + /** BLAS (Basic Linear Algebra Subprograms) instance. */ + private static final BLAS blas = BLAS.getInstance(); + + /** + * Constructs a new instance of dataset wrapper that delegates {@code compute} actions to the actual delegate. + * + * @param delegate delegate that performs {@code compute} actions + */ + AlgorithmSpecificDataset( + Dataset delegate) { + super(delegate); + } + + /** Calculate gradient. */ + double[] gradient(double[] x) { + return computeWithCtx((ctx, data, partIdx) -> { + double[] tmp = Arrays.copyOf(data.getLabels(), data.getRows()); + blas.dgemv("N", data.getRows(), data.getCols(), 1.0, data.getFeatures(), + Math.max(1, data.getRows()), x, 1, -1.0, tmp, 1); + + double[] res = new double[data.getCols()]; + blas.dgemv("T", data.getRows(), data.getCols(), 1.0, data.getFeatures(), + Math.max(1, data.getRows()), tmp, 1, 0.0, res, 1); + + int iteration = ctx.getIteration(); + + System.out.println("Iteration " + iteration + " on partition " + partIdx + + " completed with local result " + Arrays.toString(res)); + + ctx.setIteration(iteration + 1); + + return res; + }, this::sum); + } + + /** Sum of two vectors. */ + public double[] sum(double[] a, double[] b) { + if (a == null) + return b; + + if (b == null) + return a; + + blas.daxpy(a.length, 1.0, a, 1, b, 1); + + return b; + } + } + + /** Algorithm specific partition context. */ + private static class AlgorithmSpecificPartitionContext implements Serializable { + /** */ + private static final long serialVersionUID = 1887368924266684044L; + + /** Iteration number. */ + private int iteration; + + /** */ + public int getIteration() { + return iteration; + } + + /** */ + public void setIteration(int iteration) { + this.iteration = iteration; + } + } + + /** */ + private static IgniteCache createCache(Ignite ignite) { + CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); + cacheConfiguration.setName("PERSONS"); + cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 2)); + IgniteCache persons = ignite.createCache(cacheConfiguration); + persons.put(1, new Person("Mike", 1, 1)); + persons.put(2, new Person("John", 2, 2)); + persons.put(3, new Person("George", 3, 3)); + persons.put(4, new Person("Karl", 4, 4)); + return persons; + } +} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/CacheBasedDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/CacheBasedDatasetExample.java index fafead51bdb9c..adae6615a9065 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/CacheBasedDatasetExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/CacheBasedDatasetExample.java @@ -23,11 +23,12 @@ import org.apache.ignite.Ignition; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.examples.ml.dataset.model.Person; import org.apache.ignite.ml.dataset.DatasetFactory; import org.apache.ignite.ml.dataset.api.SimpleDataset; /** - * How to create a DLC dataset from an existing Ignite Cache? + * How to create a cache based dataset from an existing Ignite Cache? */ public class CacheBasedDatasetExample { /** Run example. */ @@ -37,17 +38,11 @@ public static void main(String[] args) { IgniteCache persons = createCache(ignite); - // Initialization of the d-learn context. After this step context cache will be created with partitions - // placed on the same nodes as the upstream Ignite Cache. - - // Loading of the d-learn context. During this step data will be transferred from the upstream cache to - // context cache with specified transformation (it will be performed locally because partitions are on the - // same nodes). In this case for every partition in upstream cache will be created labeled dataset partition - // and this new partition will be filled with help of specified feature and label extractors. + // Creates a cache based simple dataset containing features and providing standard dataset API. SimpleDataset dataset = DatasetFactory.createSimpleDataset( ignite, persons, - (k, v) -> new double[]{ v.age, v.salary }, + (k, v) -> new double[]{ v.getAge(), v.getSalary() }, 2 ); @@ -87,23 +82,4 @@ private static IgniteCache createCache(Ignite ignite) { persons.put(4, new Person("Karl", 24, 70000)); return persons; } - - /** */ - private static class Person { - /** */ - private final String name; - - /** */ - private final double age; - - /** */ - private final double salary; - - /** */ - public Person(String name, double age, double salary) { - this.name = name; - this.age = age; - this.salary = salary; - } - } } diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalBasedDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalDatasetExample.java similarity index 72% rename from examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalBasedDatasetExample.java rename to examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalDatasetExample.java index 2d4a436a99a1a..4c563c2b2cc78 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalBasedDatasetExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalDatasetExample.java @@ -22,13 +22,14 @@ import java.util.Map; import org.apache.ignite.Ignite; import org.apache.ignite.Ignition; +import org.apache.ignite.examples.ml.dataset.model.Person; import org.apache.ignite.ml.dataset.DatasetFactory; import org.apache.ignite.ml.dataset.api.SimpleDataset; /** - * How to create a DLC dataset from an existing local data? + * How to create a local dataset from an existing local data? */ -public class LocalBasedDatasetExample { +public class LocalDatasetExample { /** Run example. */ public static void main(String[] args) { try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { @@ -36,18 +37,11 @@ public static void main(String[] args) { Map persons = createCache(ignite); - // Initialization of the d-learn context. After this step context cache will be created with partitions - // placed on the same nodes as the upstream Ignite Cache. - - // Loading of the d-learn context. During this step data will be transferred from the upstream cache to - // context cache with specified transformation (it will be performed locally because partitions are on the - // same nodes). In this case for every partition in upstream cache will be created labeled dataset partition - // and this new partition will be filled with help of specified feature and label extractors. - + // Creates a local simple dataset containing features and providing standard dataset API. SimpleDataset dataset = DatasetFactory.createSimpleDataset( persons, 2, - (k, v) -> new double[]{ v.age, v.salary }, + (k, v) -> new double[]{ v.getAge(), v.getSalary() }, 2 ); @@ -84,23 +78,4 @@ private static Map createCache(Ignite ignite) { persons.put(4, new Person("Karl", 24, 70000)); return persons; } - - /** */ - private static class Person { - /** */ - private final String name; - - /** */ - private final double age; - - /** */ - private final double salary; - - /** */ - public Person(String name, double age, double salary) { - this.name = name; - this.age = age; - this.salary = salary; - } - } } diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/model/Person.java b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/model/Person.java new file mode 100644 index 0000000000000..54a59d040bee9 --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/model/Person.java @@ -0,0 +1,58 @@ +/* + * 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.examples.ml.dataset.model; + +/** Person model. */ +public class Person { + /** Name. */ + private final String name; + + /** Age. */ + private final double age; + + /** Salary. */ + private final double salary; + + /** + * Constructs a new instance of person. + * + * @param name name + * @param age age + * @param salary salary + */ + public Person(String name, double age, double salary) { + this.name = name; + this.age = age; + this.salary = salary; + } + + /** */ + public String getName() { + return name; + } + + /** */ + public double getAge() { + return age; + } + + /** */ + public double getSalary() { + return salary; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java index ef43bd55e6769..17ccd8246790f 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java @@ -145,7 +145,13 @@ public double[][] corr() { return cov; } - /** */ + /** + * Returns the sum of the two specified vectors. Be aware that it is in-place operation. + * + * @param a first vector + * @param b second vector + * @return sum of the two specified vectors + */ private static double[] sum(double[] a, double[] b) { for (int i = 0; i < a.length; i++) a[i] += b[i]; @@ -153,7 +159,13 @@ private static double[] sum(double[] a, double[] b) { return a; } - /** */ + /** + * Returns the sum of the two specified matrices. Be aware that it is in-place operation. + * + * @param a first matrix + * @param b second matrix + * @return sum of the two specified matrices + */ private static double[][] sum(double[][] a, double[][] b) { for (int i = 0; i < a.length; i++) for (int j = 0; j < a[i].length; j++) @@ -162,7 +174,14 @@ private static double[][] sum(double[][] a, double[][] b) { return a; } - /** */ + /** + * Multiplies all elements of the specified matrix on specified multiplier {@code alpha}. Be aware that it is + * in-place operation. + * + * @param a matrix to be scaled + * @param alpha multiplier + * @return scaled matrix + */ private static double[][] scale(double[][] a, double alpha) { for (int i = 0; i < a.length; i++) for (int j = 0; j < a[i].length; j++) @@ -171,15 +190,24 @@ private static double[][] scale(double[][] a, double alpha) { return a; } - /** */ + /** + * Util class that keeps values and count of rows this value has been calculated on. + * + * @param type of a value + */ private static class ValueWithCount { - /** */ + /** Value. */ private final V val; - /** */ + /** Count of rows the value has been calculated on. */ private final int cnt; - /** */ + /** + * Constructs a new instance of value with count. + * + * @param val value + * @param cnt count of rows the value has been calculated on + */ ValueWithCount(V val, int cnt) { this.val = val; this.cnt = cnt; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java index cb052e9ec811e..7a5c3d4a47673 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java @@ -101,7 +101,12 @@ public CacheBasedDataset(Ignite ignite, IgniteCache upstreamCache, partDataBuilder ); - return map.apply(ctx, data, part); + R res = map.apply(ctx, data, part); + + // Saves partition context after update. + ComputeUtils.saveContext(Ignition.localIgnite(), datasetCacheName, part, ctx); + + return res; }, reduce, identity); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java index e092b77bbff61..340e24e2df791 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java @@ -230,14 +230,27 @@ public static void initContext(Ignite ignite, Str * Extracts partition {@code context} from the Ignite Cache. * * @param ignite Ignite instance - * @param datasetCacheName collection of cache names + * @param datasetCacheName dataset cache names * @param part partition index * @param type of a partition {@code context} - * @return + * @return partition {@code context} */ public static C getContext(Ignite ignite, String datasetCacheName, int part) { - IgniteCache learningCtxCache = ignite.cache(datasetCacheName); - return learningCtxCache.get(part); + IgniteCache datasetCache = ignite.cache(datasetCacheName); + return datasetCache.get(part); + } + + /** + * Saves the specified partition {@code context} into the Ignite Cache. + * + * @param ignite Ignite instance + * @param datasetCacheName dataset cache name + * @param part partition index + * @param type of a partition {@code context} + */ + public static void saveContext(Ignite ignite, String datasetCacheName, int part, C ctx) { + IgniteCache datasetCache = ignite.cache(datasetCacheName); + datasetCache.put(part, ctx); } /** From 616c18e52b4bcd44b47f13c1eab37bcf16df3259 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Mon, 29 Jan 2018 14:20:37 +0300 Subject: [PATCH 22/38] IGNITE-7437 Add package-info.java files. --- .../ml/dataset/model/package-info.java | 22 +++++++++++++++++++ .../examples/ml/dataset/package-info.java | 2 +- .../api/builder/context/package-info.java | 22 +++++++++++++++++++ .../api/builder/data/package-info.java | 22 +++++++++++++++++++ .../ml/dataset/api/builder/package-info.java | 22 +++++++++++++++++++ .../ml/dataset/api/context/package-info.java | 22 +++++++++++++++++++ .../ml/dataset/api/data/package-info.java | 22 +++++++++++++++++++ .../ignite/ml/dataset/api/package-info.java | 22 +++++++++++++++++++ .../ml/dataset/impl/cache/package-info.java | 22 +++++++++++++++++++ .../dataset/impl/cache/util/package-info.java | 22 +++++++++++++++++++ .../ml/dataset/impl/local/package-info.java | 22 +++++++++++++++++++ .../ignite/ml/dataset/impl/package-info.java | 22 +++++++++++++++++++ .../ignite/ml/dataset/package-info.java | 22 +++++++++++++++++++ 13 files changed, 265 insertions(+), 1 deletion(-) create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dataset/model/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/package-info.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/model/package-info.java b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/model/package-info.java new file mode 100644 index 0000000000000..86df33274aaa1 --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/model/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Models used in machine learning dataset examples. + */ +package org.apache.ignite.examples.ml.dataset.model; \ No newline at end of file diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/package-info.java b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/package-info.java index 325dc1d7991b6..2d0fc1d58b067 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/package-info.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/package-info.java @@ -17,6 +17,6 @@ /** * - * Machine learning Distributed Learning Context framework examples. + * Machine learning dataset examples. */ package org.apache.ignite.examples.ml.dataset; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/package-info.java new file mode 100644 index 0000000000000..166a02da98baa --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Contains partition {@code context} builders. + */ +package org.apache.ignite.ml.dataset.api.builder.context; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/package-info.java new file mode 100644 index 0000000000000..a019c762873be --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Contains partition {@code data} builders. + */ +package org.apache.ignite.ml.dataset.api.builder.data; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/package-info.java new file mode 100644 index 0000000000000..55310426b8d27 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Base package for partition {@code data} and {@code context} builders. + */ +package org.apache.ignite.ml.dataset.api.builder; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/package-info.java new file mode 100644 index 0000000000000..4f376d9569240 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Contains implementation of partition {@code context}. + */ +package org.apache.ignite.ml.dataset.api.context; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/package-info.java new file mode 100644 index 0000000000000..83eca8be8d7a0 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Contains implementation of partition {@code data}. + */ +package org.apache.ignite.ml.dataset.api.data; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/package-info.java new file mode 100644 index 0000000000000..19c051c00cc22 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Base package for machine learning dataset classes representing dataset API. + */ +package org.apache.ignite.ml.dataset.api; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/package-info.java new file mode 100644 index 0000000000000..74629d7ddfa25 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Base package for cache based implementation of machine learning dataset. + */ +package org.apache.ignite.ml.dataset.impl.cache; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/package-info.java new file mode 100644 index 0000000000000..89e248f0b37fd --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Contains util classes used in cache based implementation of dataset. + */ +package org.apache.ignite.ml.dataset.impl.cache.util; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/package-info.java new file mode 100644 index 0000000000000..2b1b19554eaae --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Base package for local implementation of machine learning dataset. + */ +package org.apache.ignite.ml.dataset.impl.local; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/package-info.java new file mode 100644 index 0000000000000..031a56a58999a --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Base package for implementations of machine learning dataset. + */ +package org.apache.ignite.ml.dataset.impl; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/package-info.java new file mode 100644 index 0000000000000..96a63a7e20591 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Base package for machine learning dataset classes. + */ +package org.apache.ignite.ml.dataset; \ No newline at end of file From ef092a6c1893199cb5542eabdd20d05b30327a6a Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Mon, 29 Jan 2018 14:22:24 +0300 Subject: [PATCH 23/38] IGNITE-7437 Revert wrong changes. --- .../ignite/examples/misc/deployment/DeploymentExample.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/examples/src/main/java/org/apache/ignite/examples/misc/deployment/DeploymentExample.java b/examples/src/main/java/org/apache/ignite/examples/misc/deployment/DeploymentExample.java index b42b0c991a30d..9497ed7b0e879 100644 --- a/examples/src/main/java/org/apache/ignite/examples/misc/deployment/DeploymentExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/misc/deployment/DeploymentExample.java @@ -38,7 +38,7 @@ * Demonstrates how to explicitly deploy a task. Note that * it is very rare when you would need such functionality as tasks are * auto-deployed on demand first time you execute them. So in most cases - * you would just transform any of the {@code Ignite.execute(...)} methods directly. + * you would just apply any of the {@code Ignite.execute(...)} methods directly. * However, sometimes a task is not in local class path, so you may not even * know the code it will execute, but you still need to execute it. For example, * you have two independent components in the system, and one loads the task @@ -73,7 +73,7 @@ public static void main(String[] args) throws Exception { // onto remote nodes on demand. For this example this task is // available on the classpath, however in real life that may not // always be the case. In those cases you should use explicit - // 'IgniteCompute.localDeployTask(Class, ClassLoader) transform and + // 'IgniteCompute.localDeployTask(Class, ClassLoader) apply and // then use 'IgniteCompute.execute(String, Object)' method // passing your task name as first parameter. ignite.compute().localDeployTask(ExampleTask.class, ExampleTask.class.getClassLoader()); @@ -102,7 +102,7 @@ public static void main(String[] args) throws Exception { * Example task used to demonstrate direct task deployment through API. * For this example this task as available on the classpath, however * in real life that may not always be the case. In those cases - * you should use explicit {@link IgniteCompute#localDeployTask(Class, ClassLoader)} transform and + * you should use explicit {@link IgniteCompute#localDeployTask(Class, ClassLoader)} apply and * then use {@link IgniteCompute#execute(String, Object)} * method passing your task name as first parameter. *

From ad411dbe0b039fe63c43b25774357a4a95d3a91e Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Tue, 30 Jan 2018 16:45:05 +0300 Subject: [PATCH 24/38] IGNITE-7437 Update unit tests for cache based and local datasets. --- modules/ml/pom.xml | 7 + .../dataset/impl/cache/CacheBasedDataset.java | 4 +- .../impl/cache/CacheBasedDatasetBuilder.java | 4 +- .../dataset/impl/cache/util/ComputeUtils.java | 45 +-- .../cache/util/UpstreamCursorAdapter.java | 14 +- .../ml/dataset/api/DatasetWrapperTest.java | 83 +++++ .../cache/CacheBasedDatasetBuilderTest.java | 23 +- .../impl/cache/CacheBasedDatasetTest.java | 50 ++- .../impl/cache/util/ComputeUtilsTest.java | 309 ++++++++++++++++++ .../DatasetAffinityFunctionWrapperTest.java | 110 +++++++ .../cache/util/PartitionDataStorageTest.java | 49 +++ .../impl/local/LocalDatasetBuilderTest.java | 89 +++++ .../dataset/impl/local/LocalDatasetTest.java | 4 - 13 files changed, 727 insertions(+), 64 deletions(-) create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtilsTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/DatasetAffinityFunctionWrapperTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorageTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetTest.java diff --git a/modules/ml/pom.xml b/modules/ml/pom.xml index a20a5964155b0..6e66551800f05 100644 --- a/modules/ml/pom.xml +++ b/modules/ml/pom.xml @@ -98,6 +98,13 @@ SparseBitSet 1.0 + + + org.mockito + mockito-all + 1.9.5 + test + diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java index 7a5c3d4a47673..cbf6f80fa4c72 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java @@ -44,10 +44,10 @@ public class CacheBasedDataset implements Dataset { /** Number of retries for the case when one of partitions not found on the node where computation is performed. */ - private static final int RETRIES = 100; + private static final int RETRIES = 15 * 60; /** Retry interval (ms) for the case when one of partitions not found on the node where computation is performed. */ - private static final int RETRY_INTERVAL = 500; + private static final int RETRY_INTERVAL = 1000; /** Ignite instance. */ private final Ignite ignite; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java index 2c64d542343bb..7f3407243aa14 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java @@ -41,10 +41,10 @@ public class CacheBasedDatasetBuilder implements DatasetBuilder { /** Number of retries for the case when one of partitions not found on the node where loading is performed. */ - private static final int RETRIES = 100; + private static final int RETRIES = 15 * 60; /** Retry interval (ms) for the case when one of partitions not found on the node where loading is performed. */ - private static final int RETRY_INTERVAL = 500; + private static final int RETRY_INTERVAL = 1000; /** Template of the name of Ignite Cache containing partition {@code context}. */ private static final String DATASET_CACHE_TEMPLATE = "%s_DATASET_%s"; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java index 340e24e2df791..0c7801a24e6b8 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java @@ -29,12 +29,12 @@ import javax.cache.Cache; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; import org.apache.ignite.Ignition; import org.apache.ignite.cache.affinity.Affinity; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.cluster.ClusterGroup; -import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.ml.dataset.PartitionContextBuilder; import org.apache.ignite.ml.dataset.PartitionDataBuilder; @@ -74,7 +74,7 @@ public static Collection affinityCallWithRetries(Ignite ignite, Collectio BitSet completionFlags = new BitSet(partitions); Collection results = new ArrayList<>(); - for (int t = 0; t < retries; t++) { + for (int t = 0; t <= retries; t++) { ClusterGroup clusterGrp = ignite.cluster().forDataNodes(primaryCache); // Sends jobs. @@ -83,11 +83,10 @@ public static Collection affinityCallWithRetries(Ignite ignite, Collectio if (!completionFlags.get(part)) { final int currPart = part; - futures.put(currPart, ignite.compute(clusterGrp).affinityCallAsync(cacheNames, currPart, () -> { - checkAllPartitionsAvailable(Ignition.localIgnite(), cacheNames, currPart); - - return fun.apply(currPart); - })); + futures.put( + currPart, + ignite.compute(clusterGrp).affinityCallAsync(cacheNames, currPart, () -> fun.apply(currPart)) + ); } // Collects results. @@ -97,7 +96,7 @@ public static Collection affinityCallWithRetries(Ignite ignite, Collectio results.add(res); completionFlags.set(part); } - catch (PartitionNotFoundException ignore) { + catch (IgniteException ignore) { } if (completionFlags.cardinality() == partitions) @@ -129,7 +128,8 @@ public static Collection affinityCallWithRetries(Ignite ignite, Collectio /** * Extracts partition {@code data} from the local storage, if it's not found in local storage recovers this {@code - * data} from a partition {@code upstream} and {@code context}. + * data} from a partition {@code upstream} and {@code context}. Be aware that this method should be called from + * the node where partition is placed. * * @param ignite Ignite instance * @param upstreamCacheName name of an {@code upstream} cache @@ -163,10 +163,9 @@ public static D getData( qry.setLocal(true); qry.setPartition(part); - // TODO: how to guarantee that cache size will not be changed between these calls? long cnt = upstreamCache.localSizeLong(part); try (QueryCursor> cursor = upstreamCache.query(qry)) { - return partDataBuilder.build(new UpstreamCursorAdapter<>(cursor.iterator()), cnt, ctx); + return partDataBuilder.build(new UpstreamCursorAdapter<>(cursor.iterator(), cnt), cnt, ctx); } }); @@ -198,7 +197,7 @@ public static void initContext(Ignite ignite, Str long cnt = locUpstreamCache.localSizeLong(part); C ctx; try (QueryCursor> cursor = locUpstreamCache.query(qry)) { - ctx = ctxBuilder.build(new UpstreamCursorAdapter<>(cursor.iterator()), cnt); + ctx = ctxBuilder.build(new UpstreamCursorAdapter<>(cursor.iterator(), cnt), cnt); } IgniteCache datasetCache = locIgnite.cache(datasetCacheName); @@ -252,26 +251,4 @@ public static void saveContext(Ignite ignite, String da IgniteCache datasetCache = ignite.cache(datasetCacheName); datasetCache.put(part, ctx); } - - /** - * Checks that partitions with the specified partition index of all caches are placed on the same node. In case of - * rebalancing it's not guaranteed that partitions of caches even with the same affinity function will be moved - * synchronously. A workaround used here is based on optimistic locking with checking that partitions available on - * the node. - * - * @param ignite Ignite instance - * @param cacheNames collection of cache names - * @param part partition index - */ - private static void checkAllPartitionsAvailable(Ignite ignite, Collection cacheNames, int part) { - for (String cacheName : cacheNames) { - Affinity affinity = ignite.affinity(cacheName); - - ClusterNode partNode = affinity.mapPartitionToNode(part); - ClusterNode locNode = ignite.cluster().localNode(); - - if (!partNode.equals(locNode)) - throw new PartitionNotFoundException(cacheName, locNode.id(), part); - } - } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java index 19ff7ea3930ab..96ec1c195c7bd 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java @@ -18,6 +18,7 @@ package org.apache.ignite.ml.dataset.impl.cache.util; import java.util.Iterator; +import java.util.NoSuchElementException; import javax.cache.Cache; import org.apache.ignite.ml.dataset.UpstreamEntry; @@ -32,22 +33,31 @@ public class UpstreamCursorAdapter implements Iterator /** Cache entry iterator. */ private final Iterator> delegate; + /** Size. */ + private long cnt; + /** * Constructs a new instance of iterator. * * @param delegate cache entry iterator */ - UpstreamCursorAdapter(Iterator> delegate) { + UpstreamCursorAdapter(Iterator> delegate, long cnt) { this.delegate = delegate; + this.cnt = cnt; } /** {@inheritDoc} */ @Override public boolean hasNext() { - return delegate.hasNext(); + return delegate.hasNext() && cnt > 0; } /** {@inheritDoc} */ @Override public UpstreamEntry next() { + if (cnt == 0) + throw new NoSuchElementException(); + + cnt--; + Cache.Entry next = delegate.next(); if (next == null) diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/api/DatasetWrapperTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/api/DatasetWrapperTest.java index 9efd630797a93..ca5cf4ca98865 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/api/DatasetWrapperTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/api/DatasetWrapperTest.java @@ -1,4 +1,87 @@ +/* + * 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.ml.dataset.api; +import java.io.Serializable; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; +import org.apache.ignite.ml.math.functions.IgniteTriFunction; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +/** + * Tests for {@link DatasetWrapper}. + */ +@RunWith(MockitoJUnitRunner.class) public class DatasetWrapperTest { + /** Mocked dataset. */ + @Mock + private Dataset dataset; + + /** Dataset wrapper. */ + private DatasetWrapper wrapper; + + /** Initialization. */ + @Before + public void beforeTest() { + wrapper = new DatasetWrapper<>(dataset); + } + + /** Tests {@code computeWithCtx()} method. */ + @Test + @SuppressWarnings("unchecked") + public void testComputeWithCtx() { + doReturn(42).when(dataset).computeWithCtx(any(IgniteTriFunction.class), any(), any()); + + Integer res = wrapper.computeWithCtx(mock(IgniteTriFunction.class), mock(IgniteBinaryOperator.class), null); + + assertEquals(42, res.intValue()); + verify(dataset, times(1)).computeWithCtx(any(IgniteTriFunction.class), any(), any()); + } + + /** Tests {@code compute()} method. */ + @Test + @SuppressWarnings("unchecked") + public void testCompute() { + doReturn(42).when(dataset).compute(any(IgniteBiFunction.class), any(), any()); + + Integer res = wrapper.compute(mock(IgniteBiFunction.class), mock(IgniteBinaryOperator.class), null); + + assertEquals(42, res.intValue()); + verify(dataset, times(1)).compute(any(IgniteBiFunction.class), any(), any()); + } + + /** Tests {@code close()} method. */ + @Test + public void testClose() throws Exception { + wrapper.close(); + + verify(dataset, times(1)).close(); + } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilderTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilderTest.java index 75accf62ce9fa..24f8612fa92e0 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilderTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilderTest.java @@ -1,3 +1,20 @@ +/* + * 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.ml.dataset.impl.cache; import java.util.Collection; @@ -15,7 +32,7 @@ * Tests for {@link CacheBasedDatasetBuilder}. */ public class CacheBasedDatasetBuilderTest extends GridCommonAbstractTest { - /** Number of nodes in grid */ + /** Number of nodes in grid. */ private static final int NODE_COUNT = 10; /** Ignite instance. */ @@ -32,9 +49,7 @@ public class CacheBasedDatasetBuilderTest extends GridCommonAbstractTest { stopAllGrids(); } - /** - * {@inheritDoc} - */ + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { /* Grid instance. */ ignite = grid(NODE_COUNT); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java index 4e5f7475a8b05..d1afc969ead9d 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java @@ -1,3 +1,20 @@ +/* + * 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.ml.dataset.impl.cache; import java.util.ArrayList; @@ -23,11 +40,14 @@ import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +/** + * Tests for {@link CacheBasedDataset}. + */ public class CacheBasedDatasetTest extends GridCommonAbstractTest { - /** Number of nodes in grid */ + /** Number of nodes in grid. */ private static final int NODE_COUNT = 4; - /** */ + /** Ignite instance. */ private Ignite ignite; /** {@inheritDoc} */ @@ -41,9 +61,7 @@ public class CacheBasedDatasetTest extends GridCommonAbstractTest { stopAllGrids(); } - /** - * {@inheritDoc} - */ + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { /* Grid instance. */ ignite = grid(NODE_COUNT); @@ -52,10 +70,11 @@ public class CacheBasedDatasetTest extends GridCommonAbstractTest { } /** - * Tests that partitions of upstream cache and context cache are reserved during computations on base learning - * context. Reservation means that partitions won't be unloaded from the node before computation will be completed. + * Tests that partitions of the upstream cache and the partition {@code context} cache are reserved during + * computations on dataset. Reservation means that partitions won't be unloaded from the node before computation is + * completed. */ - public void testPartitionExchangeDuringComputeWithReturnCallOnBaseContext() { + public void testPartitionExchangeDuringComputeCall() { int partitions = 4; IgniteCache upstreamCache = generateTestData(4, 0); @@ -83,13 +102,12 @@ public void testPartitionExchangeDuringComputeWithReturnCallOnBaseContext() { computationsLock.lock(); try { - new Thread(() -> dataset.compute((part, partIndex) -> { + new Thread(() -> dataset.compute((data, partIndex) -> { // track number of started computations ignite.atomicLong(numOfStartedComputationsId.toString(), 0, false).incrementAndGet(); ignite.reentrantLock(computationsLockId.toString(), false, true, false).lock(); ignite.reentrantLock(computationsLockId.toString(), false, true, false).unlock(); - return 42; - }, (a, b) -> 42)).start(); + })).start(); // wait all computations to start while (numOfStartedComputations.get() < partitions) { @@ -107,10 +125,11 @@ public void testPartitionExchangeDuringComputeWithReturnCallOnBaseContext() { } /** - * Tests that partitions of upstream cache and context cache are reserved during computations on base learning - * context. Reservation means that partitions won't be unloaded from the node before computation will be completed. + * Tests that partitions of the upstream cache and the partition {@code context} cache are reserved during + * computations on dataset. Reservation means that partitions won't be unloaded from the node before computation is + * completed. */ - public void testPartitionExchangeDuringComputeCallOnBaseContext() { + public void testPartitionExchangeDuringComputeWithCtxCall() { int partitions = 4; IgniteCache upstreamCache = generateTestData(4, 0); @@ -125,7 +144,6 @@ public void testPartitionExchangeDuringComputeCallOnBaseContext() { CacheBasedDataset dataset = builder.build(); - assertTrue("Before computation all partitions should not be reserved", areAllPartitionsNotReserved(upstreamCache.getName(), dataset.getDatasetCache().getName())); @@ -139,7 +157,7 @@ public void testPartitionExchangeDuringComputeCallOnBaseContext() { computationsLock.lock(); try { - new Thread(() -> dataset.compute((part, partIndex) -> { + new Thread(() -> dataset.computeWithCtx((ctx, data, partIndex) -> { // track number of started computations ignite.atomicLong(numOfStartedComputationsId.toString(), 0, false).incrementAndGet(); ignite.reentrantLock(computationsLockId.toString(), false, true, false).lock(); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtilsTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtilsTest.java new file mode 100644 index 0000000000000..137f3294dca19 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtilsTest.java @@ -0,0 +1,309 @@ +/* + * 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.ml.dataset.impl.cache.util; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteAtomicLong; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.affinity.AffinityFunction; +import org.apache.ignite.cache.affinity.AffinityFunctionContext; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.ml.dataset.UpstreamEntry; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests for {@link ComputeUtils}. + */ +public class ComputeUtilsTest extends GridCommonAbstractTest { + /** Number of nodes in grid. */ + private static final int NODE_COUNT = 10; + + /** Ignite instance. */ + private Ignite ignite; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + for (int i = 1; i <= NODE_COUNT; i++) + startGrid(i); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + /* Grid instance. */ + ignite = grid(NODE_COUNT); + ignite.configuration().setPeerClassLoadingEnabled(true); + IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); + } + + /** + * Tests that in case two caches maintain their partitions on different nodes, affinity call won't be completed. + */ + public void testAffinityCallWithRetriesNegative() { + ClusterNode node1 = grid(1).cluster().localNode(); + ClusterNode node2 = grid(2).cluster().localNode(); + + String firstCacheName = "CACHE_1_" + UUID.randomUUID(); + String secondCacheName = "CACHE_2_" + UUID.randomUUID(); + + CacheConfiguration cacheConfiguration1 = new CacheConfiguration<>(); + cacheConfiguration1.setName(firstCacheName); + cacheConfiguration1.setAffinity(new TestAffinityFunction(node1)); + IgniteCache cache1 = ignite.createCache(cacheConfiguration1); + + CacheConfiguration cacheConfiguration2 = new CacheConfiguration<>(); + cacheConfiguration2.setName(secondCacheName); + cacheConfiguration2.setAffinity(new TestAffinityFunction(node2)); + IgniteCache cache2 = ignite.createCache(cacheConfiguration2); + + try { + try { + ComputeUtils.affinityCallWithRetries( + ignite, + Arrays.asList(firstCacheName, secondCacheName), + part -> part, + 0 + ); + } + catch (IllegalStateException expectedException) { + return; + } + + fail("Missing IllegalStateException"); + } + finally { + cache1.destroy(); + cache2.destroy(); + } + } + + /** + * Test that in case two caches maintain their partitions on the same node, affinity call will be completed. + */ + public void testAffinityCallWithRetriesPositive() { + ClusterNode node = grid(1).cluster().localNode(); + + String firstCacheName = "CACHE_1_" + UUID.randomUUID(); + String secondCacheName = "CACHE_2_" + UUID.randomUUID(); + + CacheConfiguration cacheConfiguration1 = new CacheConfiguration<>(); + cacheConfiguration1.setName(firstCacheName); + cacheConfiguration1.setAffinity(new TestAffinityFunction(node)); + IgniteCache cache1 = ignite.createCache(cacheConfiguration1); + + CacheConfiguration cacheConfiguration2 = new CacheConfiguration<>(); + cacheConfiguration2.setName(secondCacheName); + cacheConfiguration2.setAffinity(new TestAffinityFunction(node)); + IgniteCache cache2 = ignite.createCache(cacheConfiguration2); + + try (IgniteAtomicLong cnt = ignite.atomicLong("COUNTER_" + UUID.randomUUID(), 0, true)) { + + ComputeUtils.affinityCallWithRetries(ignite, Arrays.asList(firstCacheName, secondCacheName), part -> { + Ignite locIgnite = Ignition.localIgnite(); + + assertEquals(node, locIgnite.cluster().localNode()); + + cnt.incrementAndGet(); + + return part; + }, 0); + + assertEquals(1, cnt.get()); + } + finally { + cache1.destroy(); + cache2.destroy(); + } + } + + /** + * Tests {@code getData()} method. + */ + public void testGetData() { + ClusterNode node = grid(1).cluster().localNode(); + + String upstreamCacheName = "CACHE_1_" + UUID.randomUUID(); + String datasetCacheName = "CACHE_2_" + UUID.randomUUID(); + + CacheConfiguration upstreamCacheConfiguration = new CacheConfiguration<>(); + upstreamCacheConfiguration.setName(upstreamCacheName); + upstreamCacheConfiguration.setAffinity(new TestAffinityFunction(node)); + IgniteCache upstreamCache = ignite.createCache(upstreamCacheConfiguration); + + CacheConfiguration datasetCacheConfiguration = new CacheConfiguration<>(); + datasetCacheConfiguration.setName(datasetCacheName); + datasetCacheConfiguration.setAffinity(new TestAffinityFunction(node)); + IgniteCache datasetCache = ignite.createCache(datasetCacheConfiguration); + + upstreamCache.put(42, 42); + datasetCache.put(0, 0); + + UUID datasetId = UUID.randomUUID(); + + IgniteAtomicLong cnt = ignite.atomicLong("CNT_" + datasetId, 0, true); + + for (int i = 0; i < 10; i++) { + Collection data = ComputeUtils.affinityCallWithRetries( + ignite, + Arrays.asList(datasetCacheName, upstreamCacheName), + part -> ComputeUtils.getData( + ignite, + upstreamCacheName, + datasetCacheName, + datasetId, + 0, + (upstream, upstreamSize, ctx) -> { + cnt.incrementAndGet(); + + assertEquals(1, upstreamSize); + + UpstreamEntry e = upstream.next(); + return new TestPartitionData(e.getKey() + e.getValue()); + } + ), + 0 + ); + + assertEquals(1, data.size()); + + TestPartitionData dataElement = data.iterator().next(); + assertEquals(84, dataElement.val.intValue()); + } + + assertEquals(1, cnt.get()); + } + + /** + * Tests {@code initContext()} method. + */ + public void testInitContext() { + ClusterNode node = grid(1).cluster().localNode(); + + String upstreamCacheName = "CACHE_1_" + UUID.randomUUID(); + String datasetCacheName = "CACHE_2_" + UUID.randomUUID(); + + CacheConfiguration upstreamCacheConfiguration = new CacheConfiguration<>(); + upstreamCacheConfiguration.setName(upstreamCacheName); + upstreamCacheConfiguration.setAffinity(new TestAffinityFunction(node)); + IgniteCache upstreamCache = ignite.createCache(upstreamCacheConfiguration); + + CacheConfiguration datasetCacheConfiguration = new CacheConfiguration<>(); + datasetCacheConfiguration.setName(datasetCacheName); + datasetCacheConfiguration.setAffinity(new TestAffinityFunction(node)); + IgniteCache datasetCache = ignite.createCache(datasetCacheConfiguration); + + upstreamCache.put(42, 42); + + ComputeUtils.initContext( + ignite, + upstreamCacheName, + datasetCacheName, + (upstream, upstreamSize) -> { + + assertEquals(1, upstreamSize); + + UpstreamEntry e = upstream.next(); + return e.getKey() + e.getValue(); + }, + 0 + ); + + assertEquals(1, datasetCache.size()); + assertEquals(84, datasetCache.get(0).intValue()); + } + + /** + * Test partition data. + */ + private static class TestPartitionData implements AutoCloseable { + /** Value. */ + private final Integer val; + + /** + * Constructs a new instance of test partition data. + * + * @param val values + */ + TestPartitionData(Integer val) { + this.val = val; + } + + /** {@inheritDoc} */ + @Override public void close() throws Exception { + // Do nothing, GC will clean up. + } + } + + /** + * Affinity function used in tests in this class. Defines one partition and assign it on the specified cluster node. + */ + private static class TestAffinityFunction implements AffinityFunction { + /** */ + private static final long serialVersionUID = -1353725303983563094L; + + /** Cluster node partition will be assigned on. */ + private final ClusterNode node; + + /** + * Constructs a new instance of test affinity function. + * + * @param node cluster node partition will be assigned on + */ + TestAffinityFunction(ClusterNode node) { + this.node = node; + } + + /** {@inheritDoc} */ + @Override public void reset() { + // Do nothing. + } + + /** {@inheritDoc} */ + @Override public int partitions() { + return 1; + } + + /** {@inheritDoc} */ + @Override public int partition(Object key) { + return 0; + } + + /** {@inheritDoc} */ + @Override public List> assignPartitions(AffinityFunctionContext affCtx) { + return Collections.singletonList(Collections.singletonList(node)); + } + + /** {@inheritDoc} */ + @Override public void removeNode(UUID nodeId) { + // Do nothing. + } + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/DatasetAffinityFunctionWrapperTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/DatasetAffinityFunctionWrapperTest.java new file mode 100644 index 0000000000000..2628aa6991fcf --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/DatasetAffinityFunctionWrapperTest.java @@ -0,0 +1,110 @@ +/* + * 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.ml.dataset.impl.cache.util; + +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import org.apache.ignite.cache.affinity.AffinityFunction; +import org.apache.ignite.cache.affinity.AffinityFunctionContext; +import org.apache.ignite.cluster.ClusterNode; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +/** + * Tests for {@link DatasetAffinityFunctionWrapper}. + */ +@RunWith(MockitoJUnitRunner.class) +public class DatasetAffinityFunctionWrapperTest { + /** Mocked affinity function. */ + @Mock + private AffinityFunction affinityFunction; + + /** Wrapper. */ + private DatasetAffinityFunctionWrapper wrapper; + + /** Initialization. */ + @Before + public void beforeTest() { + wrapper = new DatasetAffinityFunctionWrapper(affinityFunction); + } + + /** Tests {@code reset()} method. */ + @Test + public void testReset() { + wrapper.reset(); + + verify(affinityFunction, times(1)).reset(); + } + + /** Tests {@code partitions()} method. */ + @Test + public void testPartitions() { + doReturn(42).when(affinityFunction).partitions(); + + int partitions = wrapper.partitions(); + + assertEquals(42, partitions); + verify(affinityFunction, times(1)).partitions(); + } + + /** Tests {@code partition} method. */ + @Test + public void testPartition() { + doReturn(0).when(affinityFunction).partition(eq(42)); + + int part = wrapper.partition(42); + + assertEquals(42, part); + verify(affinityFunction, times(0)).partition(any()); + } + + /** Tests {@code assignPartitions()} method. */ + @Test + public void testAssignPartitions() { + List> nodes = Collections.singletonList(Collections.singletonList(mock(ClusterNode.class))); + + doReturn(nodes).when(affinityFunction).assignPartitions(any()); + + List> resNodes = wrapper.assignPartitions(mock(AffinityFunctionContext.class)); + + assertEquals(nodes, resNodes); + verify(affinityFunction, times(1)).assignPartitions(any()); + } + + /** Tests {@code removeNode()} method. */ + @Test + public void testRemoveNode() { + UUID nodeId = UUID.randomUUID(); + + wrapper.removeNode(nodeId); + + verify(affinityFunction, times(1)).removeNode(eq(nodeId)); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorageTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorageTest.java new file mode 100644 index 0000000000000..eab2be1bec8ff --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorageTest.java @@ -0,0 +1,49 @@ +/* + * 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.ml.dataset.impl.cache.util; + +import java.util.concurrent.atomic.AtomicLong; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link PartitionDataStorage}. + */ +public class PartitionDataStorageTest { + /** Data storage. */ + private PartitionDataStorage dataStorage = new PartitionDataStorage(); + + /** Tests {@code computeDataIfAbsent()} method. */ + @Test + public void testComputeDataIfAbsent() { + AtomicLong cnt = new AtomicLong(); + + for (int i = 0; i < 10; i++) { + Integer res = (Integer) dataStorage.computeDataIfAbsent(0, () -> { + cnt.incrementAndGet(); + + return 42; + }); + + assertEquals(42, res.intValue()); + } + + assertEquals(1, cnt.intValue()); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilderTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilderTest.java index 193000380b6f9..7c5887e01f25b 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilderTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilderTest.java @@ -1,4 +1,93 @@ +/* + * 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.ml.dataset.impl.local; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link LocalDatasetBuilder}. + */ public class LocalDatasetBuilderTest { + /** Tests {@code build()} method. */ + @Test + public void testBuild() { + Map data = new HashMap<>(); + for (int i = 0; i < 100; i++) + data.put(i, i); + + LocalDatasetBuilder builder = new LocalDatasetBuilder<>( + data, + 10, + (upstream, upstreamSize) -> null, + (upstream, upstreamSize, ctx) -> { + int[] arr = new int[Math.toIntExact(upstreamSize)]; + + int ptr = 0; + while (upstream.hasNext()) + arr[ptr++] = upstream.next().getValue(); + + return new TestPartitionData(arr); + } + ); + + LocalDataset dataset = builder.build(); + + AtomicLong cnt = new AtomicLong(); + + dataset.compute((partData, partIdx) -> { + cnt.incrementAndGet(); + + int[] arr = partData.data; + + assertEquals(10, arr.length); + + for (int i = 0; i < 10; i++) + assertEquals(partIdx * 10 + i, arr[i]); + }); + + assertEquals(10, cnt.intValue()); + } + + /** + * Test partition {@code data}. + */ + private static class TestPartitionData implements AutoCloseable { + /** Data. */ + private int[] data; + + /** + * Constructs a new test partition data instance. + * + * @param data data + */ + TestPartitionData(int[] data) { + this.data = data; + } + + /** {@inheritDoc} */ + @Override public void close() throws Exception { + // Do nothing, GC will clean up. + } + } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetTest.java deleted file mode 100644 index b7b873ed5e9b0..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetTest.java +++ /dev/null @@ -1,4 +0,0 @@ -package org.apache.ignite.ml.dataset.impl.local; - -public class LocalDatasetTest { -} From 91f612a22c03a07bda7addcbc6a1e0a00098f33c Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Tue, 30 Jan 2018 16:48:13 +0300 Subject: [PATCH 25/38] IGNITE-7437 Revert wrong changes. --- .../trees/trainers/columnbased/ColumnDecisionTreeTrainer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/ColumnDecisionTreeTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/ColumnDecisionTreeTrainer.java index ef703846026da..fec0a83fd503c 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/ColumnDecisionTreeTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/ColumnDecisionTreeTrainer.java @@ -172,7 +172,7 @@ private static class IndexAndSplitInfo { } /** - * Utility class used to withDataBuilder decision tree. Basically it is pointer to leaf node. + * Utility class used to build decision tree. Basically it is pointer to leaf node. */ private static class TreeTip { /** */ From 189df16331e59eeaa678cd9d1eebe7a720ee2535 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Tue, 30 Jan 2018 16:49:12 +0300 Subject: [PATCH 26/38] IGNITE-7437 Revert wrong changes. --- .../org/apache/ignite/ml/DLCPlayground.java | 73 ------------------- 1 file changed, 73 deletions(-) delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java b/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java deleted file mode 100644 index 766417b72d7d4..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/DLCPlayground.java +++ /dev/null @@ -1,73 +0,0 @@ -package org.apache.ignite.ml; - -import java.util.Arrays; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.dataset.DatasetFactory; -import org.apache.ignite.ml.dataset.api.SimpleDataset; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -/** */ -public class DLCPlayground extends GridCommonAbstractTest { - /** Number of nodes in grid */ - private static final int NODE_COUNT = 4; - - /** */ - private Ignite ignite; - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - for (int i = 1; i <= NODE_COUNT; i++) - startGrid(i); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() { - stopAllGrids(); - } - - /** - * {@inheritDoc} - */ - @Override protected void beforeTest() throws Exception { - /* Grid instance. */ - ignite = grid(NODE_COUNT); - ignite.configuration().setPeerClassLoadingEnabled(true); - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - } - - /** */ - public void testTest() { - CacheConfiguration cc = new CacheConfiguration<>(); - cc.setAffinity(new RendezvousAffinityFunction(false, 2)); - cc.setName("TEST"); - IgniteCache cache = ignite.createCache(cc); - for (int i = 0; i < 40; i++) - cache.put(i, i); - - SimpleDataset dataset = DatasetFactory.createSimpleDataset(ignite, cache, (k, v) -> new double[] { 42.0 }, 1); - - // Calculation of the mean value. This calculation will be performed in map-reduce manner. - double[] mean = dataset.mean(); - System.out.println("Mean \n\t" + Arrays.toString(mean)); - - // Calculation of the standard deviation. This calculation will be performed in map-reduce manner. - double[] std = dataset.std(); - System.out.println("Standard deviation \n\t" + Arrays.toString(std)); - - // Calculation of the covariance matrix. This calculation will be performed in map-reduce manner. - double[][] cov = dataset.cov(); - System.out.println("Covariance matrix "); - for (double[] row : cov) - System.out.println("\t" + Arrays.toString(row)); - - // Calculation of the correlation matrix. This calculation will be performed in map-reduce manner. - double[][] corr = dataset.corr(); - System.out.println("Correlation matrix "); - for (double[] row : corr) - System.out.println("\t" + Arrays.toString(row)); - } -} From 39e22b44bb54019ad501a6f7669b8ec85272a0d5 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Tue, 30 Jan 2018 17:27:37 +0300 Subject: [PATCH 27/38] IGNITE-7437 Update javadoc in accordance with code style. --- .../AlgorithmSpecificDatasetExample.java | 2 +- .../examples/ml/dataset/model/Person.java | 6 +- .../org/apache/ignite/ml/dataset/Dataset.java | 90 ++++----- .../ignite/ml/dataset/DatasetBuilder.java | 6 +- .../ignite/ml/dataset/DatasetFactory.java | 172 +++++++++--------- .../ml/dataset/PartitionContextBuilder.java | 18 +- .../ml/dataset/PartitionDataBuilder.java | 28 +-- .../ignite/ml/dataset/UpstreamEntry.java | 8 +- .../ignite/ml/dataset/api/DatasetWrapper.java | 6 +- .../ignite/ml/dataset/api/SimpleDataset.java | 36 ++-- .../ml/dataset/api/SimpleLabeledDataset.java | 4 +- .../builder/context/EmptyContextBuilder.java | 4 +- .../data/SimpleDatasetDataBuilder.java | 10 +- .../data/SimpleLabeledDatasetDataBuilder.java | 12 +- .../dataset/api/data/SimpleDatasetData.java | 6 +- .../api/data/SimpleLabeledDatasetData.java | 8 +- .../dataset/impl/cache/CacheBasedDataset.java | 28 +-- .../impl/cache/CacheBasedDatasetBuilder.java | 16 +- .../dataset/impl/cache/util/ComputeUtils.java | 96 +++++----- .../util/DatasetAffinityFunctionWrapper.java | 6 +- .../impl/cache/util/PartitionDataStorage.java | 6 +- .../util/PartitionNotFoundException.java | 46 ----- .../cache/util/UpstreamCursorAdapter.java | 6 +- .../ml/dataset/impl/local/LocalDataset.java | 11 +- .../impl/local/LocalDatasetBuilder.java | 26 +-- .../cache/CacheBasedDatasetBuilderTest.java | 6 +- .../impl/cache/CacheBasedDatasetTest.java | 16 +- .../impl/cache/util/ComputeUtilsTest.java | 4 +- .../impl/local/LocalDatasetBuilderTest.java | 2 +- 29 files changed, 320 insertions(+), 365 deletions(-) delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionNotFoundException.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/AlgorithmSpecificDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/AlgorithmSpecificDatasetExample.java index 61a531e8ab350..4d07c46d67f69 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/AlgorithmSpecificDatasetExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/AlgorithmSpecificDatasetExample.java @@ -94,7 +94,7 @@ private static class AlgorithmSpecificDataset /** * Constructs a new instance of dataset wrapper that delegates {@code compute} actions to the actual delegate. * - * @param delegate delegate that performs {@code compute} actions + * @param delegate Delegate that performs {@code compute} actions. */ AlgorithmSpecificDataset( Dataset delegate) { diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/model/Person.java b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/model/Person.java index 54a59d040bee9..3770de815ed11 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/model/Person.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/model/Person.java @@ -31,9 +31,9 @@ public class Person { /** * Constructs a new instance of person. * - * @param name name - * @param age age - * @param salary salary + * @param name Name. + * @param age Age. + * @param salary Salary. */ public Person(String name, double age, double salary) { this.name = name; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/Dataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/Dataset.java index 4471cd900875c..24a206368ddb5 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/Dataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/Dataset.java @@ -44,8 +44,8 @@ * should be {@link AutoCloseable} to allow system to clean up correspondent resources when partition {@code data} is * not needed anymore. * - * @param type of a partition {@code context} - * @param type of a partition {@code data} + * @param Type of a partition {@code context}. + * @param Type of a partition {@code data}. * * @see CacheBasedDataset * @see LocalDataset @@ -56,11 +56,11 @@ public interface Dataset extend * Applies the specified {@code map} function to every partition {@code data}, {@code context} and partition * index in the dataset and then reduces {@code map} results to final result by using the {@code reduce} function. * - * @param map function applied to every partition {@code data}, {@code context} and partition index - * @param reduce function applied to results of {@code map} to get final result - * @param identity identity - * @param type of a result - * @return final result + * @param map Function applied to every partition {@code data}, {@code context} and partition index. + * @param reduce Function applied to results of {@code map} to get final result. + * @param identity Identity. + * @param Type of a result. + * @return Final result. */ public R computeWithCtx(IgniteTriFunction map, IgniteBinaryOperator reduce, R identity); @@ -68,11 +68,11 @@ public interface Dataset extend * Applies the specified {@code map} function to every partition {@code data} and partition index in the dataset * and then reduces {@code map} results to final result by using the {@code reduce} function. * - * @param map function applied to every partition {@code data} and partition index - * @param reduce function applied to results of {@code map} to get final result - * @param identity identity - * @param type of a result - * @return final result + * @param map Function applied to every partition {@code data} and partition index. + * @param reduce Function applied to results of {@code map} to get final result. + * @param identity Identity. + * @param Type of a result. + * @return Final result. */ public R compute(IgniteBiFunction map, IgniteBinaryOperator reduce, R identity); @@ -80,10 +80,10 @@ public interface Dataset extend * Applies the specified {@code map} function to every partition {@code data}, {@code context} and partition * index in the dataset and then reduces {@code map} results to final result by using the {@code reduce} function. * - * @param map function applied to every partition {@code data}, {@code context} and partition index - * @param reduce function applied to results of {@code map} to get final result - * @param type of a result - * @return final result + * @param map Function applied to every partition {@code data}, {@code context} and partition index. + * @param reduce Function applied to results of {@code map} to get final result. + * @param Type of a result. + * @return Final result. */ default public R computeWithCtx(IgniteTriFunction map, IgniteBinaryOperator reduce) { return computeWithCtx(map, reduce, null); @@ -93,10 +93,10 @@ default public R computeWithCtx(IgniteTriFunction map, Ign * Applies the specified {@code map} function to every partition {@code data} and partition index in the dataset * and then reduces {@code map} results to final result by using the {@code reduce} function. * - * @param map function applied to every partition {@code data} and partition index - * @param reduce function applied to results of {@code map} to get final result - * @param type of a result - * @return final result + * @param map Function applied to every partition {@code data} and partition index. + * @param reduce Function applied to results of {@code map} to get final result. + * @param Type of a result. + * @return Final result. */ default public R compute(IgniteBiFunction map, IgniteBinaryOperator reduce) { return compute(map, reduce, null); @@ -106,11 +106,11 @@ default public R compute(IgniteBiFunction map, IgniteBinaryOp * Applies the specified {@code map} function to every partition {@code data} and {@code context} in the dataset * and then reduces {@code map} results to final result by using the {@code reduce} function. * - * @param map function applied to every partition {@code data} and {@code context} - * @param reduce function applied to results of {@code map} to get final result - * @param identity identity - * @param type of a result - * @return final result + * @param map Function applied to every partition {@code data} and {@code context}. + * @param reduce Function applied to results of {@code map} to get final result. + * @param identity Identity. + * @param Type of a result. + * @return Final result. */ default public R computeWithCtx(IgniteBiFunction map, IgniteBinaryOperator reduce, R identity) { return computeWithCtx((ctx, data, partIdx) -> map.apply(ctx, data), reduce, identity); @@ -120,11 +120,11 @@ default public R computeWithCtx(IgniteBiFunction map, IgniteBinaryO * Applies the specified {@code map} function to every partition {@code data} in the dataset and then reduces * {@code map} results to final result by using the {@code reduce} function. * - * @param map function applied to every partition {@code data} - * @param reduce function applied to results of {@code map} to get final result - * @param identity identity - * @param type of a result - * @return final result + * @param map Function applied to every partition {@code data}. + * @param reduce Function applied to results of {@code map} to get final result. + * @param identity Identity. + * @param Type of a result. + * @return Final result. */ default public R compute(IgniteFunction map, IgniteBinaryOperator reduce, R identity) { return compute((data, partIdx) -> map.apply(data), reduce, identity); @@ -134,10 +134,10 @@ default public R compute(IgniteFunction map, IgniteBinaryOperator r * Applies the specified {@code map} function to every partition {@code data} and {@code context} in the dataset * and then reduces {@code map} results to final result by using the {@code reduce} function. * - * @param map function applied to every partition {@code data} and {@code context} - * @param reduce function applied to results of {@code map} to get final result - * @param type of a result - * @return final result + * @param map Function applied to every partition {@code data} and {@code context}. + * @param reduce Function applied to results of {@code map} to get final result. + * @param Type of a result. + * @return Final result. */ default public R computeWithCtx(IgniteBiFunction map, IgniteBinaryOperator reduce) { return computeWithCtx((ctx, data, partIdx) -> map.apply(ctx, data), reduce); @@ -147,10 +147,10 @@ default public R computeWithCtx(IgniteBiFunction map, IgniteBinaryO * Applies the specified {@code map} function to every partition {@code data} in the dataset and then reduces * {@code map} results to final result by using the {@code reduce} function. * - * @param map function applied to every partition {@code data} - * @param reduce function applied to results of {@code map} to get final result - * @param type of a result - * @return final result + * @param map Function applied to every partition {@code data}. + * @param reduce Function applied to results of {@code map} to get final result. + * @param Type of a result. + * @return Final result. */ default public R compute(IgniteFunction map, IgniteBinaryOperator reduce) { return compute((data, partIdx) -> map.apply(data), reduce); @@ -160,7 +160,7 @@ default public R compute(IgniteFunction map, IgniteBinaryOperator r * Applies the specified {@code map} function to every partition {@code data}, {@code context} and partition * index in the dataset. * - * @param map function applied to every partition {@code data}, {@code context} and partition index + * @param map Function applied to every partition {@code data}, {@code context} and partition index. */ default public void computeWithCtx(IgniteTriConsumer map) { computeWithCtx((ctx, data, partIdx) -> { @@ -172,7 +172,7 @@ default public void computeWithCtx(IgniteTriConsumer map) { /** * Applies the specified {@code map} function to every partition {@code data} in the dataset and partition index. * - * @param map function applied to every partition {@code data} and partition index + * @param map Function applied to every partition {@code data} and partition index. */ default public void compute(IgniteBiConsumer map) { compute((data, partIdx) -> { @@ -184,7 +184,7 @@ default public void compute(IgniteBiConsumer map) { /** * Applies the specified {@code map} function to every partition {@code data} and {@code context} in the dataset. * - * @param map function applied to every partition {@code data} and {@code context} + * @param map Function applied to every partition {@code data} and {@code context}. */ default public void computeWithCtx(IgniteBiConsumer map) { computeWithCtx((ctx, data, partIdx) -> map.accept(ctx, data)); @@ -193,7 +193,7 @@ default public void computeWithCtx(IgniteBiConsumer map) { /** * Applies the specified {@code map} function to every partition {@code data} in the dataset. * - * @param map function applied to every partition {@code data} + * @param map Function applied to every partition {@code data}. */ default public void compute(IgniteConsumer map) { compute((data, partIdx) -> map.accept(data)); @@ -203,9 +203,9 @@ default public void compute(IgniteConsumer map) { * Wraps this dataset into the specified wrapper to introduce new functionality based on {@code compute} and * {@code computeWithCtx} methods. * - * @param wrapper dataset wrapper - * @param type of a new wrapped dataset - * @return new wrapped dataset + * @param wrapper Dataset wrapper. + * @param Type of a new wrapped dataset. + * @return New wrapped dataset. */ default public > I wrap(IgniteFunction, I> wrapper) { return wrapper.apply(this); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java index 4bc082e5e981a..67e6b9d575e94 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java @@ -26,8 +26,8 @@ * building specific datasets such as allocation required data structures and initialization of {@code context} part of * partitions. * - * @param type of a partition {@code context} - * @param type of a partition {@code data} + * @param Type of a partition {@code context}. + * @param Type of a partition {@code data}. * * @see CacheBasedDatasetBuilder * @see LocalDatasetBuilder @@ -38,7 +38,7 @@ public interface DatasetBuilder * Constructs a new instance of {@link Dataset} that includes allocation required data structures and * initialization of {@code context} part of partitions. * - * @return dataset + * @return Dataset. */ public Dataset build(); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetFactory.java index 32ece9478ba85..f4a87281af612 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetFactory.java @@ -75,15 +75,15 @@ public class DatasetFactory { * {@code partDataBuilder}. This is the generic methods that allows to create any Ignite Cache based datasets with * any desired partition {@code context} and {@code data}. * - * @param ignite Ignite instance - * @param upstreamCache Ignite Cache with {@code upstream} data - * @param partCtxBuilder partition {@code context} builder - * @param partDataBuilder partition {@code data} builder - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @param type of a partition {@code context} - * @param type of a partition {@code data} - * @return dataset + * @param ignite Ignite instance. + * @param upstreamCache Ignite Cache with {@code upstream} data. + * @param partCtxBuilder Partition {@code context} builder. + * @param partDataBuilder Partition {@code data} builder. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @param Type of a partition {@code context}. + * @param Type of a partition {@code data}. + * @return Dataset. */ public static Dataset create( Ignite ignite, IgniteCache upstreamCache, PartitionContextBuilder partCtxBuilder, @@ -96,15 +96,15 @@ public static Dataset type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @param type of a partition {@code context} - * @return dataset + * @param ignite Ignite instance. + * @param upstreamCache Ignite Cache with {@code upstream} data. + * @param partCtxBuilder Partition {@code context} builder. + * @param featureExtractor Feature extractor used to extract features and build {@link SimpleDatasetData}. + * @param cols Number of columns (features) will be extracted. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @param Type of a partition {@code context}. + * @return Dataset. */ public static SimpleDataset createSimpleDataset(Ignite ignite, IgniteCache upstreamCache, PartitionContextBuilder partCtxBuilder, @@ -122,16 +122,16 @@ public static SimpleDataset createSimpleDatase * {@code featureExtractor} and {@code lbExtractor}. This method determines partition {@code data} to be * {@link SimpleLabeledDatasetData}, but allows to use any desired type of partition {@code context}. * - * @param ignite Ignite instance - * @param upstreamCache Ignite Cache with {@code upstream} data - * @param partCtxBuilder partition {@code context} builder - * @param featureExtractor feature extractor used to extract features and build {@link SimpleLabeledDatasetData} - * @param lbExtractor label extractor used to extract labels and buikd {@link SimpleLabeledDatasetData} - * @param cols number of columns (features) will be extracted - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @param type of a partition {@code context} - * @return dataset + * @param ignite Ignite instance. + * @param upstreamCache Ignite Cache with {@code upstream} data. + * @param partCtxBuilder Partition {@code context} builder. + * @param featureExtractor Feature extractor used to extract features and build {@link SimpleLabeledDatasetData}. + * @param lbExtractor Label extractor used to extract labels and buikd {@link SimpleLabeledDatasetData}. + * @param cols Number of columns (features) will be extracted. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @param Type of a partition {@code context}. + * @return Dataset. */ public static SimpleLabeledDataset createSimpleLabeledDataset(Ignite ignite, IgniteCache upstreamCache, PartitionContextBuilder partCtxBuilder, @@ -149,13 +149,13 @@ public static SimpleLabeledDataset createSimpl * methods determines partition {@code context} to be {@link EmptyContext} and partition {@code data} to be * {@link SimpleDatasetData}. * - * @param ignite Ignite instance - * @param upstreamCache Ignite Cache with {@code upstream} data - * @param featureExtractor feature extractor used to extract features and build {@link SimpleDatasetData} - * @param cols number of columns (features) will be extracted - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @return dataset + * @param ignite Ignite instance. + * @param upstreamCache Ignite Cache with {@code upstream} data. + * @param featureExtractor Feature extractor used to extract features and build {@link SimpleDatasetData}. + * @param cols Number of columns (features) will be extracted. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @return Dataset. */ public static SimpleDataset createSimpleDataset(Ignite ignite, IgniteCache upstreamCache, IgniteBiFunction featureExtractor, int cols) { @@ -167,14 +167,14 @@ public static SimpleDataset createSimpleDataset(Ignite igni * and {@code lbExtractor}. This methods determines partition {@code context} to be {@link EmptyContext} and * partition {@code data} to be {@link SimpleLabeledDatasetData}. * - * @param ignite Ignite instance - * @param upstreamCache Ignite Cache with {@code upstream} data - * @param featureExtractor feature extractor used to extract features and build {@link SimpleLabeledDatasetData} - * @param lbExtractor label extractor used to extract labels and buikd {@link SimpleLabeledDatasetData} - * @param cols number of columns (features) will be extracted - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @return dataset + * @param ignite Ignite instance. + * @param upstreamCache Ignite Cache with {@code upstream} data. + * @param featureExtractor Feature extractor used to extract features and build {@link SimpleLabeledDatasetData}. + * @param lbExtractor Label extractor used to extract labels and buikd {@link SimpleLabeledDatasetData}. + * @param cols Number of columns (features) will be extracted. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @return Dataset. */ public static SimpleLabeledDataset createSimpleLabeledDataset(Ignite ignite, IgniteCache upstreamCache, IgniteBiFunction featureExtractor, @@ -188,15 +188,15 @@ public static SimpleLabeledDataset createSimpleLabeledDatas * This is the generic methods that allows to create any Ignite Cache based datasets with any desired partition * {@code context} and {@code data}. * - * @param upstreamMap {@code Map} with {@code upstream} data - * @param partitions number of partitions {@code upstream} {@code Map} will be divided on - * @param partCtxBuilder partition {@code context} builder - * @param partDataBuilder partition {@code data} builder - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @param type of a partition {@code context} - * @param type of a partition {@code data} - * @return dataset + * @param upstreamMap {@code Map} with {@code upstream} data. + * @param partitions Number of partitions {@code upstream} {@code Map} will be divided on. + * @param partCtxBuilder Partition {@code context} builder. + * @param partDataBuilder Partition {@code data} builder. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @param Type of a partition {@code context}. + * @param Type of a partition {@code data}. + * @return Dataset. */ public static Dataset create( Map upstreamMap, int partitions, PartitionContextBuilder partCtxBuilder, @@ -209,15 +209,15 @@ public static Dataset type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @param type of a partition {@code context} - * @return dataset + * @param upstreamMap {@code Map} with {@code upstream} data. + * @param partitions Number of partitions {@code upstream} {@code Map} will be divided on. + * @param partCtxBuilder Partition {@code context} builder. + * @param featureExtractor Feature extractor used to extract features and build {@link SimpleDatasetData}. + * @param cols Number of columns (features) will be extracted. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @param Type of a partition {@code context}. + * @return Dataset. */ public static SimpleDataset createSimpleDataset(Map upstreamMap, int partitions, PartitionContextBuilder partCtxBuilder, @@ -235,16 +235,16 @@ public static SimpleDataset createSimpleDatase * {@code featureExtractor} and {@code lbExtractor}. This method determines partition {@code data} to be * {@link SimpleLabeledDatasetData}, but allows to use any desired type of partition {@code context}. * - * @param upstreamMap {@code Map} with {@code upstream} data - * @param partitions number of partitions {@code upstream} {@code Map} will be divided on - * @param partCtxBuilder partition {@code context} builder - * @param featureExtractor feature extractor used to extract features and build {@link SimpleLabeledDatasetData} - * @param lbExtractor label extractor used to extract labels and buikd {@link SimpleLabeledDatasetData} - * @param cols number of columns (features) will be extracted - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @param type of a partition {@code context} - * @return dataset + * @param upstreamMap {@code Map} with {@code upstream} data. + * @param partitions Number of partitions {@code upstream} {@code Map} will be divided on. + * @param partCtxBuilder Partition {@code context} builder. + * @param featureExtractor Feature extractor used to extract features and build {@link SimpleLabeledDatasetData}. + * @param lbExtractor Label extractor used to extract labels and buikd {@link SimpleLabeledDatasetData}. + * @param cols Number of columns (features) will be extracted. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @param Type of a partition {@code context}. + * @return Dataset. */ public static SimpleLabeledDataset createSimpleLabeledDataset( Map upstreamMap, int partitions, PartitionContextBuilder partCtxBuilder, @@ -262,13 +262,13 @@ public static SimpleLabeledDataset createSimpl * methods determines partition {@code context} to be {@link EmptyContext} and partition {@code data} to be * {@link SimpleDatasetData}. * - * @param upstreamMap {@code Map} with {@code upstream} data - * @param partitions number of partitions {@code upstream} {@code Map} will be divided on - * @param featureExtractor feature extractor used to extract features and build {@link SimpleDatasetData} - * @param cols number of columns (features) will be extracted - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @return dataset + * @param upstreamMap {@code Map} with {@code upstream} data. + * @param partitions Number of partitions {@code upstream} {@code Map} will be divided on. + * @param featureExtractor Feature extractor used to extract features and build {@link SimpleDatasetData}. + * @param cols Number of columns (features) will be extracted. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @return Dataset. */ public static SimpleDataset createSimpleDataset(Map upstreamMap, int partitions, IgniteBiFunction featureExtractor, int cols) { @@ -280,14 +280,14 @@ public static SimpleDataset createSimpleDataset(Map u * and {@code lbExtractor}. This methods determines partition {@code context} to be {@link EmptyContext} and * partition {@code data} to be {@link SimpleLabeledDatasetData}. * - * @param upstreamMap {@code Map} with {@code upstream} data - * @param partitions number of partitions {@code upstream} {@code Map} will be divided on - * @param featureExtractor feature extractor used to extract features and build {@link SimpleLabeledDatasetData} - * @param lbExtractor label extractor used to extract labels and buikd {@link SimpleLabeledDatasetData} - * @param cols number of columns (features) will be extracted - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @return dataset + * @param upstreamMap {@code Map} with {@code upstream} data. + * @param partitions Number of partitions {@code upstream} {@code Map} will be divided on. + * @param featureExtractor Feature extractor used to extract features and build {@link SimpleLabeledDatasetData}. + * @param lbExtractor Label extractor used to extract labels and build {@link SimpleLabeledDatasetData}. + * @param cols Number of columns (features) will be extracted. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @return Dataset. */ public static SimpleLabeledDataset createSimpleLabeledDataset(Map upstreamMap, int partitions, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor, diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionContextBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionContextBuilder.java index a64b5d8585430..f7032c617288e 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionContextBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionContextBuilder.java @@ -27,9 +27,9 @@ * build a partition {@code context} and assumed to be called only once for every partition during a dataset * initialization. * - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @param type of a partition {@code context} + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @param Type of a partition {@code context}. * * @see EmptyContextBuilder */ @@ -38,9 +38,9 @@ public interface PartitionContextBuilder extends S /** * Builds a new partition {@code context} from an {@code upstream} data. * - * @param upstreamData partition {@code upstream} data - * @param upstreamDataSize partition {@code upstream} data size - * @return partition {@code context} + * @param upstreamData Partition {@code upstream} data. + * @param upstreamDataSize Partition {@code upstream} data size. + * @return Partition {@code context}. */ public C build(Iterator> upstreamData, long upstreamDataSize); @@ -48,9 +48,9 @@ public interface PartitionContextBuilder extends S * Makes a composed partition {@code context} builder that first builds a {@code context} and then applies the * specified function on the result. * - * @param fun function that applied after first partition {@code context} is built - * @param new type of a partition {@code context} - * @return composed partition {@code context} builder + * @param fun Function that applied after first partition {@code context} is built. + * @param New type of a partition {@code context}. + * @return Composed partition {@code context} builder. */ default public PartitionContextBuilder andThen(IgniteFunction fun) { return (upstreamData, upstreamDataSize) -> fun.apply(build(upstreamData, upstreamDataSize)); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java index 9cf1d610a8b3b..f09bc96238c8c 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/PartitionDataBuilder.java @@ -29,11 +29,10 @@ * partition {@code data} not found on the node that performs computation (it might be the result of a previous node * failure or rebalancing). * - * @param type of a key in upstream data - * @param type of a value in upstream data - * @param type of a partition context - * @param type of a partition data - * + * @param Type of a key in upstream data. + * @param Type of a value in upstream data. + * @param Type of a partition context. + * @param Type of a partition data. * @see SimpleDatasetDataBuilder * @see SimpleLabeledDatasetDataBuilder */ @@ -42,10 +41,10 @@ public interface PartitionDataBuilder> upstreamData, long upstreamDataSize, C ctx); @@ -53,11 +52,12 @@ public interface PartitionDataBuilder new type of a partition {@code data} - * @return composed partition {@code data} builder + * @param fun Function that applied after first partition {@code data} is built. + * @param New type of a partition {@code data}. + * @return Composed partition {@code data} builder. */ - default public PartitionDataBuilder andThen(IgniteBiFunction fun) { - return (upstreamData, upstreamDataSize, ctx) -> fun.apply(build(upstreamData, upstreamDataSize, ctx), ctx); + default public PartitionDataBuilder andThen( + IgniteBiFunction fun) { + return (upstreamData, upstreamDataSize, ctx) -> fun.apply(build(upstreamData, upstreamDataSize, ctx), ctx); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/UpstreamEntry.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/UpstreamEntry.java index 857a47e25424d..58226d9d06960 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/UpstreamEntry.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/UpstreamEntry.java @@ -20,8 +20,8 @@ /** * Entry of the {@code upstream}. * - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. */ public class UpstreamEntry { /** Key. */ @@ -33,8 +33,8 @@ public class UpstreamEntry { /** * Constructs a new instance of upstream entry. * - * @param key key - * @param val value + * @param key Key. + * @param val Value. */ public UpstreamEntry(K key, V val) { this.key = key; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/DatasetWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/DatasetWrapper.java index b6270dce248f8..436e45761ebb1 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/DatasetWrapper.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/DatasetWrapper.java @@ -26,8 +26,8 @@ /** * A dataset wrapper that allows to introduce new functionality based on common {@code compute} methods. * - * @param type of a partition {@code context} - * @param type of a partition {@code data} + * @param Type of a partition {@code context}. + * @param Type of a partition {@code data}. * * @see SimpleDataset * @see SimpleLabeledDataset @@ -39,7 +39,7 @@ public class DatasetWrapper imp /** * Constructs a new instance of dataset wrapper that delegates {@code compute} actions to the actual delegate. * - * @param delegate delegate that performs {@code compute} actions + * @param delegate Delegate that performs {@code compute} actions. */ public DatasetWrapper(Dataset delegate) { this.delegate = delegate; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java index 17ccd8246790f..c5bf06413e26c 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java @@ -25,7 +25,7 @@ /** * A simple dataset introduces additional methods based on a matrix of features. * - * @param type of a partition {@code context} + * @param Type of a partition {@code context}. */ public class SimpleDataset extends DatasetWrapper { /** BLAS (Basic Linear Algebra Subprograms) instance. */ @@ -34,7 +34,7 @@ public class SimpleDataset extends DatasetWrapper delegate) { super(delegate); @@ -43,7 +43,7 @@ public SimpleDataset(Dataset delegate) { /** * Calculates mean value by all columns. * - * @return mean values + * @return Mean values. */ public double[] mean() { ValueWithCount res = delegate.compute((data, partIdx) -> { @@ -71,7 +71,7 @@ public double[] mean() { /** * Calculates standard deviation by all columns. * - * @return standard deviations + * @return Standard deviations. */ public double[] std() { double[] mean = mean(); @@ -102,7 +102,7 @@ public double[] std() { /** * Calculates covariance matrix by all columns. * - * @return covariance matrix + * @return Covariance matrix. */ public double[][] cov() { double[] mean = mean(); @@ -132,7 +132,7 @@ public double[][] cov() { /** * Calculates correlation matrix by all columns. * - * @return correlation matrix + * @return Correlation matrix. */ public double[][] corr() { double[][] cov = cov(); @@ -148,9 +148,9 @@ public double[][] corr() { /** * Returns the sum of the two specified vectors. Be aware that it is in-place operation. * - * @param a first vector - * @param b second vector - * @return sum of the two specified vectors + * @param a First vector. + * @param b Second vector. + * @return Sum of the two specified vectors. */ private static double[] sum(double[] a, double[] b) { for (int i = 0; i < a.length; i++) @@ -162,9 +162,9 @@ private static double[] sum(double[] a, double[] b) { /** * Returns the sum of the two specified matrices. Be aware that it is in-place operation. * - * @param a first matrix - * @param b second matrix - * @return sum of the two specified matrices + * @param a First matrix. + * @param b Second matrix. + * @return Sum of the two specified matrices. */ private static double[][] sum(double[][] a, double[][] b) { for (int i = 0; i < a.length; i++) @@ -178,9 +178,9 @@ private static double[][] sum(double[][] a, double[][] b) { * Multiplies all elements of the specified matrix on specified multiplier {@code alpha}. Be aware that it is * in-place operation. * - * @param a matrix to be scaled - * @param alpha multiplier - * @return scaled matrix + * @param a Matrix to be scaled. + * @param alpha Multiplier. + * @return Scaled matrix. */ private static double[][] scale(double[][] a, double alpha) { for (int i = 0; i < a.length; i++) @@ -193,7 +193,7 @@ private static double[][] scale(double[][] a, double alpha) { /** * Util class that keeps values and count of rows this value has been calculated on. * - * @param type of a value + * @param Type of a value. */ private static class ValueWithCount { /** Value. */ @@ -205,8 +205,8 @@ private static class ValueWithCount { /** * Constructs a new instance of value with count. * - * @param val value - * @param cnt count of rows the value has been calculated on + * @param val Value. + * @param cnt Count of rows the value has been calculated on. */ ValueWithCount(V val, int cnt) { this.val = val; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleLabeledDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleLabeledDataset.java index 2fede65054cb6..7e6a876ab7196 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleLabeledDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleLabeledDataset.java @@ -24,14 +24,14 @@ /** * A simple labeled dataset introduces additional methods based on a matrix of features and labels vector. * - * @param type of a partition {@code context} + * @param Type of a partition {@code context}. */ public class SimpleLabeledDataset extends DatasetWrapper { /** * Creates a new instance of simple labeled dataset that introduces additional methods based on a matrix of features * and labels vector. * - * @param delegate delegate that performs {@code compute} actions + * @param delegate Delegate that performs {@code compute} actions. */ public SimpleLabeledDataset(Dataset delegate) { super(delegate); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/EmptyContextBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/EmptyContextBuilder.java index 894013dd534bd..466f7ca5a68b4 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/EmptyContextBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/EmptyContextBuilder.java @@ -25,8 +25,8 @@ /** * A partition {@code context} builder that makes {@link EmptyContext}. * - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. */ public class EmptyContextBuilder implements PartitionContextBuilder { /** */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleDatasetDataBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleDatasetDataBuilder.java index ab154a4fcfebc..95ec5a8781d35 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleDatasetDataBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleDatasetDataBuilder.java @@ -27,9 +27,9 @@ /** * A partition {@code data} builder that makes {@link SimpleDatasetData}. * - * @param type of a key in upstream data - * @param type of a value in upstream data - * @param type of a partition context + * @param Type of a key in upstream data. + * @param Type of a value in upstream data. + * @param Type of a partition context. */ public class SimpleDatasetDataBuilder implements PartitionDataBuilder { @@ -45,8 +45,8 @@ public class SimpleDatasetDataBuilder /** * Construct a new instance of partition {@code data} builder that makes {@link SimpleDatasetData}. * - * @param featureExtractor function that extracts features from an {@code upstream} data - * @param cols number of columns (features) + * @param featureExtractor Function that extracts features from an {@code upstream} data. + * @param cols Number of columns (features). */ public SimpleDatasetDataBuilder(IgniteBiFunction featureExtractor, int cols) { this.featureExtractor = featureExtractor; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleLabeledDatasetDataBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleLabeledDatasetDataBuilder.java index fa63f453336bf..0765daa027b75 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleLabeledDatasetDataBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleLabeledDatasetDataBuilder.java @@ -27,9 +27,9 @@ /** * A partition {@code data} builder that makes {@link SimpleLabeledDatasetData}. * - * @param type of a key in upstream data - * @param type of a value in upstream data - * @param type of a partition context + * @param Type of a key in upstream data. + * @param Type of a value in upstream data. + * @param type of a partition context. */ public class SimpleLabeledDatasetDataBuilder implements PartitionDataBuilder { @@ -48,9 +48,9 @@ public class SimpleLabeledDatasetDataBuilder /** * Constructs a new instance of partition {@code data} builder that makes {@link SimpleLabeledDatasetData}. * - * @param featureExtractor function that extracts features from an {@code upstream} data - * @param lbExtractor function that extracts labels from an {@code upstream} data - * @param cols number of columns (features) + * @param featureExtractor Function that extracts features from an {@code upstream} data. + * @param lbExtractor Function that extracts labels from an {@code upstream} data. + * @param cols Number of columns (features). */ public SimpleLabeledDatasetDataBuilder(IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor, int cols) { diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleDatasetData.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleDatasetData.java index d174c50588b43..60d599cbbf730 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleDatasetData.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleDatasetData.java @@ -37,9 +37,9 @@ public class SimpleDatasetData implements AutoCloseable { * Constructs a new instance of partition {@code data} of the {@link SimpleDataset} containing matrix of features in * flat column-major format stored in heap. * - * @param features matrix of features in a dense flat column-major format - * @param rows number of rows - * @param cols number of columns + * @param features Matrix of features in a dense flat column-major format. + * @param rows Number of rows. + * @param cols Number of columns. */ public SimpleDatasetData(double[] features, int rows, int cols) { this.features = features; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleLabeledDatasetData.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleLabeledDatasetData.java index eb40a30d3d7ce..d326c0672c023 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleLabeledDatasetData.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleLabeledDatasetData.java @@ -40,10 +40,10 @@ public class SimpleLabeledDatasetData implements AutoCloseable { * Constructs a new instance of partition {@code data} of the {@link SimpleLabeledDataset} containing matrix of * features in flat column-major format stored in heap and vector of labels stored in heap as well. * - * @param features matrix with features in a dense flat column-major format - * @param rows number of rows - * @param cols number of columns - * @param labels vector with labels + * @param features Matrix with features in a dense flat column-major format. + * @param rows Number of rows. + * @param cols Number of columns. + * @param labels Vector with labels. */ public SimpleLabeledDatasetData(double[] features, int rows, int cols, double[] labels) { this.features = features; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java index cbf6f80fa4c72..463d4964d09b1 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java @@ -36,10 +36,10 @@ * An implementation of dataset based on Ignite Cache, which is used as {@code upstream} and as reliable storage for * partition {@code context} as well. * - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @param type of a partition {@code context} - * @param type of a partition {@code data} + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @param Type of a partition {@code context}. + * @param Type of a partition {@code data}. */ public class CacheBasedDataset implements Dataset { @@ -68,11 +68,11 @@ public class CacheBasedDataset upstreamCache, IgniteCache datasetCache, PartitionDataBuilder partDataBuilder, @@ -139,11 +139,11 @@ public CacheBasedDataset(Ignite ignite, IgniteCache upstreamCache, * partitions with guarantee that partitions with the same index of upstream and partition {@code context} caches * will be on the same node during the computation and will not be moved before computation is finished. * - * @param fun function that applies to all partitions - * @param reduce function that reduces results of {@code fun} - * @param identity identity - * @param type of a result - * @return final result + * @param fun Function that applies to all partitions. + * @param reduce Function that reduces results of {@code fun}. + * @param identity Identity. + * @param Type of a result. + * @return Final result. */ private R computeForAllPartitions(IgniteFunction fun, IgniteBinaryOperator reduce, R identity) { Collection cacheNames = Arrays.asList(datasetCache.getName(), upstreamCache.getName()); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java index 7f3407243aa14..2cf2617c70157 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java @@ -33,10 +33,10 @@ * A dataset builder that makes {@link CacheBasedDataset}. Encapsulate logic of building cache based dataset such as * allocation required data structures and initialization of {@code context} part of partitions. * - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @param type of a partition {@code context} - * @param type of a partition {@code data} + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @param Type of a partition {@code context}. + * @param Type of a partition {@code data}. */ public class CacheBasedDatasetBuilder implements DatasetBuilder { @@ -64,10 +64,10 @@ public class CacheBasedDatasetBuilder upstreamCache, PartitionContextBuilder partCtxBuilder, PartitionDataBuilder partDataBuilder) { diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java index 0c7801a24e6b8..529f982fed0fa 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java @@ -53,13 +53,13 @@ public class ComputeUtils { * finished. If partitions are placed on different nodes then call will be retried, but not more than {@code * retries} times with {@code interval} interval specified in milliseconds. * - * @param ignite Ignite instance - * @param cacheNames collection of cache names - * @param fun function to be applied on all partitions - * @param retries number of retries for the case when one of partitions not found on the node - * @param interval interval of retries for the case when one of partitions not found on the node - * @param type of a result - * @return collection of results + * @param ignite Ignite instance. + * @param cacheNames Collection of cache names. + * @param fun Function to be applied on all partitions. + * @param retries Number of retries for the case when one of partitions not found on the node. + * @param interval Interval of retries for the case when one of partitions not found on the node. + * @param Type of a result. + * @return Collection of results. */ public static Collection affinityCallWithRetries(Ignite ignite, Collection cacheNames, IgniteFunction fun, int retries, int interval) { @@ -114,12 +114,12 @@ public static Collection affinityCallWithRetries(Ignite ignite, Collectio * finished. If partitions are placed on different nodes then call will be retried, but not more than {@code * retries} times. * - * @param ignite Ignite instance - * @param cacheNames collection of cache names - * @param fun function to be applied on all partitions - * @param retries number of retries for the case when one of partitions not found on the node - * @param type of a result - * @return collection of results + * @param ignite Ignite instance. + * @param cacheNames Collection of cache names. + * @param fun Function to be applied on all partitions. + * @param retries Number of retries for the case when one of partitions not found on the node. + * @param Type of a result. + * @return Collection of results. */ public static Collection affinityCallWithRetries(Ignite ignite, Collection cacheNames, IgniteFunction fun, int retries) { @@ -131,17 +131,17 @@ public static Collection affinityCallWithRetries(Ignite ignite, Collectio * data} from a partition {@code upstream} and {@code context}. Be aware that this method should be called from * the node where partition is placed. * - * @param ignite Ignite instance - * @param upstreamCacheName name of an {@code upstream} cache - * @param datasetCacheName name of a partition {@code context} cache - * @param datasetId dataset ID - * @param part partition index - * @param partDataBuilder partition data builder - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @param type of a partition {@code context} - * @param type of a partition {@code data} - * @return partition {@code data} + * @param ignite Ignite instance. + * @param upstreamCacheName Name of an {@code upstream} cache. + * @param datasetCacheName Name of a partition {@code context} cache. + * @param datasetId Dataset ID. + * @param part Partition index. + * @param partDataBuilder Partition data builder. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @param Type of a partition {@code context}. + * @param Type of a partition {@code data}. + * @return Partition {@code data}. */ @SuppressWarnings("unchecked") public static D getData(Ignite ignite, @@ -175,13 +175,13 @@ public static D getData( /** * Initializes partition {@code context} by loading it from a partition {@code upstream}. * - * @param ignite Ignite instance - * @param upstreamCacheName name of an {@code upstream} cache - * @param datasetCacheName name of a partition {@code context} cache - * @param ctxBuilder partition {@code context} builder - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @param type of a partition {@code context} + * @param ignite Ignite instance. + * @param upstreamCacheName Name of an {@code upstream} cache. + * @param datasetCacheName Name of a partition {@code context} cache. + * @param ctxBuilder Partition {@code context} builder. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @param Type of a partition {@code context}. */ public static void initContext(Ignite ignite, String upstreamCacheName, String datasetCacheName, PartitionContextBuilder ctxBuilder, int retries, int interval) { @@ -211,14 +211,14 @@ public static void initContext(Ignite ignite, Str /** * Initializes partition {@code context} by loading it from a partition {@code upstream}. * - * @param ignite Ignite instance - * @param upstreamCacheName name of an {@code upstream} cache - * @param datasetCacheName name of a partition {@code context} cache - * @param ctxBuilder partition {@code context} builder - * @param retries number of retries for the case when one of partitions not found on the node - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @param type of a partition {@code context} + * @param ignite Ignite instance. + * @param upstreamCacheName Name of an {@code upstream} cache. + * @param datasetCacheName Name of a partition {@code context} cache. + * @param ctxBuilder Partition {@code context} builder. + * @param retries Number of retries for the case when one of partitions not found on the node. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @param Type of a partition {@code context}. */ public static void initContext(Ignite ignite, String upstreamCacheName, String datasetCacheName, PartitionContextBuilder ctxBuilder, int retries) { @@ -228,11 +228,11 @@ public static void initContext(Ignite ignite, Str /** * Extracts partition {@code context} from the Ignite Cache. * - * @param ignite Ignite instance - * @param datasetCacheName dataset cache names - * @param part partition index - * @param type of a partition {@code context} - * @return partition {@code context} + * @param ignite Ignite instance. + * @param datasetCacheName Dataset cache names. + * @param part Partition index. + * @param Type of a partition {@code context}. + * @return Partition {@code context}. */ public static C getContext(Ignite ignite, String datasetCacheName, int part) { IgniteCache datasetCache = ignite.cache(datasetCacheName); @@ -242,10 +242,10 @@ public static C getContext(Ignite ignite, String datase /** * Saves the specified partition {@code context} into the Ignite Cache. * - * @param ignite Ignite instance - * @param datasetCacheName dataset cache name - * @param part partition index - * @param type of a partition {@code context} + * @param ignite Ignite instance. + * @param datasetCacheName Dataset cache name. + * @param part Partition index. + * @param Type of a partition {@code context}. */ public static void saveContext(Ignite ignite, String datasetCacheName, int part, C ctx) { IgniteCache datasetCache = ignite.cache(datasetCacheName); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/DatasetAffinityFunctionWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/DatasetAffinityFunctionWrapper.java index 32aeb6258f026..a8f682686cd5f 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/DatasetAffinityFunctionWrapper.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/DatasetAffinityFunctionWrapper.java @@ -37,7 +37,7 @@ public class DatasetAffinityFunctionWrapper implements AffinityFunction { /** * Constructs a new instance of affinity function wrapper. * - * @param delegate affinity function which actually performs all methods except {@code partition()} + * @param delegate Affinity function which actually performs all methods except {@code partition()}. */ public DatasetAffinityFunctionWrapper(AffinityFunction delegate) { this.delegate = delegate; @@ -56,8 +56,8 @@ public DatasetAffinityFunctionWrapper(AffinityFunction delegate) { /** * Returns key as a partition index. * - * @param key partition index - * @return partition index + * @param key Partition index. + * @return Partition index. */ @Override public int partition(Object key) { return (Integer) key; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java index 2a636bea749cf..3885827663c8d 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java @@ -39,9 +39,9 @@ class PartitionDataStorage { * loads it using the specified {@code supplier}. Unlike {@link ConcurrentMap#computeIfAbsent(Object, Function)}, * this method guarantees that function will be called only once. * - * @param part partition index - * @param supplier partition {@code data} supplier. - * @return partition {@code data} + * @param part Partition index. + * @param supplier Partition {@code data} supplier. + * @return Partition {@code data}. */ Object computeDataIfAbsent(int part, Supplier supplier) { Object data = storage.get(part); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionNotFoundException.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionNotFoundException.java deleted file mode 100644 index 96fd2adab3cf6..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionNotFoundException.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.ml.dataset.impl.cache.util; - -import java.io.Serializable; -import java.util.UUID; -import org.apache.ignite.IgniteException; - -/** - * Exception which is thrown when partition is expected to be on the node but it isn't. In case of rebalancing it's not - * guaranteed that partitions of caches even with the same affinity function will be moved synchronously. A workaround - * used here is based on optimistic locking with checking that partitions available on the node. - */ -class PartitionNotFoundException extends IgniteException implements Serializable { - /** */ - private static final long serialVersionUID = -8891869046312827676L; - - /** templace of an exception message.. */ - private static final String MSG_TEMPLATE = "Partition %d of %s expected to be on node %s, but it isn't"; - - /** - * Constructs a new instance of an upstream partition not found exception. - * - * @param cacheName cache name - * @param nodeId node id - * @param partIdx partition index - */ - PartitionNotFoundException(String cacheName, UUID nodeId, int partIdx) { - super(String.format(MSG_TEMPLATE, partIdx, cacheName, nodeId.toString())); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java index 96ec1c195c7bd..cd119a133d261 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java @@ -26,8 +26,8 @@ * Cursor adapter used to transform {@link Cache.Entry} received from Ignite Cache query cursor into DLC-specific * {@link UpstreamEntry}. * - * @param type of an upstream value key - * @param type of an upstream value + * @param Type of an upstream value key. + * @param Type of an upstream value. */ public class UpstreamCursorAdapter implements Iterator> { /** Cache entry iterator. */ @@ -39,7 +39,7 @@ public class UpstreamCursorAdapter implements Iterator /** * Constructs a new instance of iterator. * - * @param delegate cache entry iterator + * @param delegate Cache entry iterator. */ UpstreamCursorAdapter(Iterator> delegate, long cnt) { this.delegate = delegate; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java index d0baadeb2c13a..c08b7dedaba17 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java @@ -28,8 +28,8 @@ * An implementation of dataset based on local data structures such as {@code Map} and {@code List} and doesn't requires * Ignite environment. Introduces for testing purposes mostly, but can be used for simple local computations as well. * - * @param type of a partition {@code context} - * @param type of a partition {@code data} + * @param Type of a partition {@code context}. + * @param Type of a partition {@code data}. */ public class LocalDataset implements Dataset { /** Partition {@code context} storage. */ @@ -42,8 +42,8 @@ public class LocalDataset imple * Constructs a new instance of dataset based on local data structures such as {@code Map} and {@code List} and * doesn't requires Ignite environment. * - * @param ctx partition {@code context} storage - * @param data partition {@code data} storage + * @param ctx Partition {@code context} storage. + * @param data Partition {@code data} storage. */ LocalDataset(List ctx, List data) { this.ctx = ctx; @@ -51,7 +51,8 @@ public class LocalDataset imple } /** {@inheritDoc} */ - @Override public R computeWithCtx(IgniteTriFunction map, IgniteBinaryOperator reduce, R identity) { + @Override public R computeWithCtx(IgniteTriFunction map, IgniteBinaryOperator reduce, + R identity) { R res = identity; for (int part = 0; part < ctx.size(); part++) diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java index 387d754dafd1d..7f85bbb2684cb 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java @@ -32,10 +32,10 @@ * A dataset builder that makes {@link LocalDataset}. Encapsulate logic of building local dataset such as allocation * required data structures and initialization of {@code context} part of partitions. * - * @param type of a key in {@code upstream} data - * @param type of a value in {@code upstream} data - * @param type of a partition {@code context} - * @param type of a partition {@code data} + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @param Type of a partition {@code context}. + * @param Type of a partition {@code data}. */ public class LocalDatasetBuilder implements DatasetBuilder { @@ -54,10 +54,10 @@ public class LocalDatasetBuilder upstreamMap, int partitions, PartitionContextBuilder partCtxBuilder, PartitionDataBuilder partDataBuilder) { @@ -103,8 +103,8 @@ public LocalDatasetBuilder(Map upstreamMap, int partitions, * Utils class that wraps iterator so that it produces only specified number of entries and allows to transform * entries from one type to another. * - * @param initial type of entries - * @param target type of entries + * @param Initial type of entries. + * @param Target type of entries. */ private static class IteratorWindow implements Iterator { /** Delegate iterator. */ @@ -122,9 +122,9 @@ private static class IteratorWindow implements Iterator { /** * Constructs a new instance of iterator window wrapper. * - * @param delegate delegate iterator - * @param map transformer that transforms entries from one type to another - * @param cnt count of entries to produce + * @param delegate Delegate iterator. + * @param map Transformer that transforms entries from one type to another. + * @param cnt Count of entries to produce. */ IteratorWindow(Iterator delegate, IgniteFunction map, int cnt) { this.delegate = delegate; diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilderTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilderTest.java index 24f8612fa92e0..de940364040f6 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilderTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilderTest.java @@ -91,9 +91,9 @@ public void testBuild() { /** * Generate an Ignite Cache with the specified size and number of partitions for testing purposes. * - * @param size size of an Ignite Cache - * @param parts number of partitions - * @return Ignite Cache instance + * @param size Size of an Ignite Cache. + * @param parts Number of partitions. + * @return Ignite Cache instance. */ private IgniteCache createTestCache(int size, int parts) { CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java index d1afc969ead9d..9695482716ba2 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java @@ -182,8 +182,8 @@ public void testPartitionExchangeDuringComputeWithCtxCall() { /** * Checks that all partitions of all specified caches are not reserved. * - * @param cacheNames cache names to be checked - * @return {@code true} if all partitions are not reserved, otherwise {@code false} + * @param cacheNames Cache names to be checked. + * @return {@code true} if all partitions are not reserved, otherwise {@code false}. */ private boolean areAllPartitionsNotReserved(String... cacheNames) { return checkAllPartitions(partition -> partition.reservations() == 0, cacheNames); @@ -192,8 +192,8 @@ private boolean areAllPartitionsNotReserved(String... cacheNames) { /** * Checks that all partitions of all specified caches not reserved. * - * @param cacheNames cache names to be checked - * @return {@code true} if all partitions are reserved, otherwise {@code false} + * @param cacheNames Cache names to be checked. + * @return {@code true} if all partitions are reserved, otherwise {@code false}. */ private boolean areAllPartitionsReserved(String... cacheNames) { return checkAllPartitions(partition -> partition.reservations() != 0, cacheNames); @@ -202,9 +202,9 @@ private boolean areAllPartitionsReserved(String... cacheNames) { /** * Checks that all partitions of all specified caches satisfies the given predicate. * - * @param pred predicate - * @param cacheNames cache names - * @return {@code true} if all partitions satisfies the given predicate + * @param pred Predicate. + * @param cacheNames Cache names. + * @return {@code true} if all partitions satisfies the given predicate. */ private boolean checkAllPartitions(IgnitePredicate pred, String... cacheNames) { boolean flag = false; @@ -340,7 +340,7 @@ public List getParts() { /** * Generates Ignite Cache with data for tests. * - * @return Ignite Cache with data for tests + * @return Ignite Cache with data for tests. */ private IgniteCache generateTestData(int partitions, int backups) { CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtilsTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtilsTest.java index 137f3294dca19..4926a909e597b 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtilsTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtilsTest.java @@ -250,7 +250,7 @@ private static class TestPartitionData implements AutoCloseable { /** * Constructs a new instance of test partition data. * - * @param val values + * @param val Value. */ TestPartitionData(Integer val) { this.val = val; @@ -275,7 +275,7 @@ private static class TestAffinityFunction implements AffinityFunction { /** * Constructs a new instance of test affinity function. * - * @param node cluster node partition will be assigned on + * @param node Cluster node partition will be assigned on. */ TestAffinityFunction(ClusterNode node) { this.node = node; diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilderTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilderTest.java index 7c5887e01f25b..f526c11101045 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilderTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilderTest.java @@ -79,7 +79,7 @@ private static class TestPartitionData implements AutoCloseable { /** * Constructs a new test partition data instance. * - * @param data data + * @param data Data. */ TestPartitionData(int[] data) { this.data = data; From 5c70866f6955b9af85cb31cf812877fb4356be1e Mon Sep 17 00:00:00 2001 From: Artem Malykh Date: Wed, 31 Jan 2018 12:47:52 +0300 Subject: [PATCH 28/38] IGNITE-7437: Small refactoring. --- .../ignite/ml/dataset/impl/cache/util/ComputeUtils.java | 4 +--- .../ml/dataset/impl/cache/util/PartitionDataStorage.java | 5 +++-- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java index 529f982fed0fa..4d45cd30e64f8 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java @@ -153,7 +153,7 @@ public static D getData( .nodeLocalMap() .computeIfAbsent(String.format(DATA_STORAGE_KEY_TEMPLATE, datasetId), key -> new PartitionDataStorage()); - Object data = dataStorage.computeDataIfAbsent(part, () -> { + return dataStorage.computeDataIfAbsent(part, () -> { IgniteCache learningCtxCache = ignite.cache(datasetCacheName); C ctx = learningCtxCache.get(part); @@ -168,8 +168,6 @@ public static D getData( return partDataBuilder.build(new UpstreamCursorAdapter<>(cursor.iterator(), cnt), cnt, ctx); } }); - - return (D)data; } /** diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java index 3885827663c8d..631bd565305ce 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java @@ -39,11 +39,12 @@ class PartitionDataStorage { * loads it using the specified {@code supplier}. Unlike {@link ConcurrentMap#computeIfAbsent(Object, Function)}, * this method guarantees that function will be called only once. * + * @param Type of data. * @param part Partition index. * @param supplier Partition {@code data} supplier. * @return Partition {@code data}. */ - Object computeDataIfAbsent(int part, Supplier supplier) { + D computeDataIfAbsent(int part, Supplier supplier) { Object data = storage.get(part); if (data == null) { @@ -58,6 +59,6 @@ Object computeDataIfAbsent(int part, Supplier supplier) { } } - return data; + return (D)data; } } From a35c04a5c60511a41bf87d79e5b272c9da09666e Mon Sep 17 00:00:00 2001 From: Artem Malykh Date: Wed, 31 Jan 2018 12:53:40 +0300 Subject: [PATCH 29/38] IGNITE-7437: Suppress 'unchecked'. IGNITE-7437: Suppress 'unchecked'. --- .../apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java | 1 - .../ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java index 4d45cd30e64f8..0785db28711bb 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java @@ -143,7 +143,6 @@ public static Collection affinityCallWithRetries(Ignite ignite, Collectio * @param Type of a partition {@code data}. * @return Partition {@code data}. */ - @SuppressWarnings("unchecked") public static D getData(Ignite ignite, String upstreamCacheName, String datasetCacheName, UUID datasetId, int part, PartitionDataBuilder partDataBuilder) { diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java index 631bd565305ce..d5c47eee7f973 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/PartitionDataStorage.java @@ -44,6 +44,7 @@ class PartitionDataStorage { * @param supplier Partition {@code data} supplier. * @return Partition {@code data}. */ + @SuppressWarnings("unchecked") D computeDataIfAbsent(int part, Supplier supplier) { Object data = storage.get(part); From 82a60f2c756d7445296621d5bc866908361762b1 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Wed, 31 Jan 2018 15:14:15 +0300 Subject: [PATCH 30/38] IGNITE-7437 Simplify dataset builder. --- .../ignite/ml/dataset/DatasetBuilder.java | 13 +++++++--- .../ignite/ml/dataset/DatasetFactory.java | 4 +-- .../impl/cache/CacheBasedDatasetBuilder.java | 21 +++------------ .../impl/local/LocalDatasetBuilder.java | 21 +++------------ .../cache/CacheBasedDatasetBuilderTest.java | 15 +++++------ .../impl/cache/CacheBasedDatasetTest.java | 26 +++++++------------ .../impl/local/LocalDatasetBuilderTest.java | 8 +++--- 7 files changed, 38 insertions(+), 70 deletions(-) diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java index 67e6b9d575e94..a6757ff12891a 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetBuilder.java @@ -26,19 +26,24 @@ * building specific datasets such as allocation required data structures and initialization of {@code context} part of * partitions. * - * @param Type of a partition {@code context}. - * @param Type of a partition {@code data}. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. * * @see CacheBasedDatasetBuilder * @see LocalDatasetBuilder * @see Dataset */ -public interface DatasetBuilder { +public interface DatasetBuilder { /** * Constructs a new instance of {@link Dataset} that includes allocation required data structures and * initialization of {@code context} part of partitions. * + * @param partCtxBuilder Partition {@code context} builder. + * @param partDataBuilder Partition {@code data} builder. + * @param Type of a partition {@code context}. + * @param Type of a partition {@code data}. * @return Dataset. */ - public Dataset build(); + public Dataset build( + PartitionContextBuilder partCtxBuilder, PartitionDataBuilder partDataBuilder); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetFactory.java index f4a87281af612..9b0ab52909759 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetFactory.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/DatasetFactory.java @@ -88,7 +88,7 @@ public class DatasetFactory { public static Dataset create( Ignite ignite, IgniteCache upstreamCache, PartitionContextBuilder partCtxBuilder, PartitionDataBuilder partDataBuilder) { - return new CacheBasedDatasetBuilder<>(ignite, upstreamCache, partCtxBuilder, partDataBuilder).build(); + return new CacheBasedDatasetBuilder<>(ignite, upstreamCache).build(partCtxBuilder, partDataBuilder); } /** @@ -201,7 +201,7 @@ public static SimpleLabeledDataset createSimpleLabeledDatas public static Dataset create( Map upstreamMap, int partitions, PartitionContextBuilder partCtxBuilder, PartitionDataBuilder partDataBuilder) { - return new LocalDatasetBuilder<>(upstreamMap, partitions, partCtxBuilder, partDataBuilder).build(); + return new LocalDatasetBuilder<>(upstreamMap, partitions).build(partCtxBuilder, partDataBuilder); } /** diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java index 2cf2617c70157..5c0d583f779a3 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilder.java @@ -35,11 +35,8 @@ * * @param Type of a key in {@code upstream} data. * @param Type of a value in {@code upstream} data. - * @param Type of a partition {@code context}. - * @param Type of a partition {@code data}. */ -public class CacheBasedDatasetBuilder - implements DatasetBuilder { +public class CacheBasedDatasetBuilder implements DatasetBuilder { /** Number of retries for the case when one of partitions not found on the node where loading is performed. */ private static final int RETRIES = 15 * 60; @@ -55,31 +52,21 @@ public class CacheBasedDatasetBuilder upstreamCache; - /** Partition {@code context} builder. */ - private final PartitionContextBuilder partCtxBuilder; - - /** Partition {@code data} builder. */ - private final PartitionDataBuilder partDataBuilder; - /** * Constructs a new instance of cache based dataset builder that makes {@link CacheBasedDataset}. * * @param ignite Ignite instance. * @param upstreamCache Ignite Cache with {@code upstream} data. - * @param partCtxBuilder Ignite Cache with partition {@code context}. - * @param partDataBuilder Partition {@code data} builder. */ - public CacheBasedDatasetBuilder(Ignite ignite, IgniteCache upstreamCache, - PartitionContextBuilder partCtxBuilder, PartitionDataBuilder partDataBuilder) { + public CacheBasedDatasetBuilder(Ignite ignite, IgniteCache upstreamCache) { this.ignite = ignite; this.upstreamCache = upstreamCache; - this.partCtxBuilder = partCtxBuilder; - this.partDataBuilder = partDataBuilder; } /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public CacheBasedDataset build() { + @Override public CacheBasedDataset build( + PartitionContextBuilder partCtxBuilder, PartitionDataBuilder partDataBuilder) { UUID datasetId = UUID.randomUUID(); // Retrieves affinity function of the upstream Ignite Cache. diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java index 7f85bbb2684cb..42c3cccdffcff 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java @@ -34,42 +34,29 @@ * * @param Type of a key in {@code upstream} data. * @param Type of a value in {@code upstream} data. - * @param Type of a partition {@code context}. - * @param Type of a partition {@code data}. */ -public class LocalDatasetBuilder - implements DatasetBuilder { +public class LocalDatasetBuilder implements DatasetBuilder { /** {@code Map} with upstream data. */ private final Map upstreamMap; /** Number of partitions. */ private final int partitions; - /** Partition {@code context} builder. */ - private final PartitionContextBuilder partCtxBuilder; - - /** Partition {@code data} builder. */ - private final PartitionDataBuilder partDataBuilder; - /** * Constructs a new instance of local dataset builder that makes {@link LocalDataset}. * * @param upstreamMap {@code Map} with upstream data. * @param partitions Number of partitions. - * @param partCtxBuilder Partition {@code context} builder. - * @param partDataBuilder Partition {@code data} builder. */ - public LocalDatasetBuilder(Map upstreamMap, int partitions, - PartitionContextBuilder partCtxBuilder, PartitionDataBuilder partDataBuilder) { + public LocalDatasetBuilder(Map upstreamMap, int partitions) { this.upstreamMap = upstreamMap; this.partitions = partitions; - this.partCtxBuilder = partCtxBuilder; - this.partDataBuilder = partDataBuilder; } /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public LocalDataset build() { + @Override public LocalDataset build( + PartitionContextBuilder partCtxBuilder, PartitionDataBuilder partDataBuilder) { List ctxList = new ArrayList<>(); List dataList = new ArrayList<>(); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilderTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilderTest.java index de940364040f6..c35cdc3239b44 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilderTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetBuilderTest.java @@ -62,15 +62,12 @@ public class CacheBasedDatasetBuilderTest extends GridCommonAbstractTest { */ public void testBuild() { IgniteCache upstreamCache = createTestCache(100, 10); - CacheBasedDatasetBuilder builder = - new CacheBasedDatasetBuilder<>( - ignite, - upstreamCache, - (upstream, upstreamSize) -> upstreamSize, - (upstream, upstreamSize, ctx) -> null - ); - - CacheBasedDataset dataset = builder.build(); + CacheBasedDatasetBuilder builder = new CacheBasedDatasetBuilder<>(ignite, upstreamCache); + + CacheBasedDataset dataset = builder.build( + (upstream, upstreamSize) -> upstreamSize, + (upstream, upstreamSize, ctx) -> null + ); Affinity upstreamAffinity = ignite.affinity(upstreamCache.getName()); Affinity datasetAffinity = ignite.affinity(dataset.getDatasetCache().getName()); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java index 9695482716ba2..f9ecb0b7aca77 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java @@ -79,15 +79,12 @@ public void testPartitionExchangeDuringComputeCall() { IgniteCache upstreamCache = generateTestData(4, 0); - CacheBasedDatasetBuilder builder = - new CacheBasedDatasetBuilder<>( - ignite, - upstreamCache, - (upstream, upstreamSize) -> upstreamSize, - (upstream, upstreamSize, ctx) -> null - ); + CacheBasedDatasetBuilder builder = new CacheBasedDatasetBuilder<>(ignite, upstreamCache); - CacheBasedDataset dataset = builder.build(); + CacheBasedDataset dataset = builder.build( + (upstream, upstreamSize) -> upstreamSize, + (upstream, upstreamSize, ctx) -> null + ); assertTrue("Before computation all partitions should not be reserved", areAllPartitionsNotReserved(upstreamCache.getName(), dataset.getDatasetCache().getName())); @@ -134,15 +131,12 @@ public void testPartitionExchangeDuringComputeWithCtxCall() { IgniteCache upstreamCache = generateTestData(4, 0); - CacheBasedDatasetBuilder builder = - new CacheBasedDatasetBuilder<>( - ignite, - upstreamCache, - (upstream, upstreamSize) -> upstreamSize, - (upstream, upstreamSize, ctx) -> null - ); + CacheBasedDatasetBuilder builder = new CacheBasedDatasetBuilder<>(ignite, upstreamCache); - CacheBasedDataset dataset = builder.build(); + CacheBasedDataset dataset = builder.build( + (upstream, upstreamSize) -> upstreamSize, + (upstream, upstreamSize, ctx) -> null + ); assertTrue("Before computation all partitions should not be reserved", areAllPartitionsNotReserved(upstreamCache.getName(), dataset.getDatasetCache().getName())); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilderTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilderTest.java index f526c11101045..0628580ce70b3 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilderTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilderTest.java @@ -36,9 +36,9 @@ public void testBuild() { for (int i = 0; i < 100; i++) data.put(i, i); - LocalDatasetBuilder builder = new LocalDatasetBuilder<>( - data, - 10, + LocalDatasetBuilder builder = new LocalDatasetBuilder<>(data, 10); + + LocalDataset dataset = builder.build( (upstream, upstreamSize) -> null, (upstream, upstreamSize, ctx) -> { int[] arr = new int[Math.toIntExact(upstreamSize)]; @@ -51,8 +51,6 @@ public void testBuild() { } ); - LocalDataset dataset = builder.build(); - AtomicLong cnt = new AtomicLong(); dataset.compute((partData, partIdx) -> { From 61a2afb655e3315d1a4a4d224a74846e40c8a6c0 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Wed, 31 Jan 2018 19:14:53 +0300 Subject: [PATCH 31/38] IGNITE-7437 Add test suite for dataset. --- .../AlgorithmSpecificDatasetExample.java | 26 +++++------ .../ml/dataset/CacheBasedDatasetExample.java | 42 ++++++++--------- .../ml/dataset/LocalDatasetExample.java | 42 ++++++++--------- .../cache/util/UpstreamCursorAdapter.java | 2 +- .../apache/ignite/ml/IgniteMLTestSuite.java | 5 ++- .../ignite/ml/dataset/DatasetTestSuite.java | 45 +++++++++++++++++++ 6 files changed, 105 insertions(+), 57 deletions(-) create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dataset/DatasetTestSuite.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/AlgorithmSpecificDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/AlgorithmSpecificDatasetExample.java index 4d07c46d67f69..654c938c5a3da 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/AlgorithmSpecificDatasetExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/AlgorithmSpecificDatasetExample.java @@ -37,14 +37,14 @@ */ public class AlgorithmSpecificDatasetExample { /** Run example. */ - public static void main(String[] args) { + public static void main(String[] args) throws Exception { try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { System.out.println(">>> D-Learn Cache Dataset example started."); IgniteCache persons = createCache(ignite); // Creates a algorithm specific dataset to perform linear regression. - AlgorithmSpecificDataset dataset = DatasetFactory.create( + try (AlgorithmSpecificDataset dataset = DatasetFactory.create( ignite, persons, (upstream, upstreamSize) -> new AlgorithmSpecificPartitionContext(), @@ -66,22 +66,22 @@ public static void main(String[] args) { return new SimpleLabeledDatasetData(a, rows, data.getCols() + 1, data.getLabels()); }) - ).wrap(AlgorithmSpecificDataset::new); + ).wrap(AlgorithmSpecificDataset::new)) { + // Trains linear regression model using gradient descent. + double[] linearRegressionMdl = new double[2]; - // Trains linear regression model using gradient descent. - double[] linearRegressionMdl = new double[2]; + for (int i = 0; i < 1000; i++) { + double[] gradient = dataset.gradient(linearRegressionMdl); - for (int i = 0; i < 1000; i++) { - double[] gradient = dataset.gradient(linearRegressionMdl); + if (BLAS.getInstance().dnrm2(gradient.length, gradient, 1) < 1e-4) + break; - if (BLAS.getInstance().dnrm2(gradient.length, gradient, 1) < 1e-4) - break; + for (int j = 0; j < gradient.length; j++) + linearRegressionMdl[j] -= 0.1 / persons.size() * gradient[j]; + } - for (int j = 0; j < gradient.length; j++) - linearRegressionMdl[j] -= 0.1 / persons.size() * gradient[j]; + System.out.println("Linear Regression Model: " + Arrays.toString(linearRegressionMdl)); } - - System.out.println("Linear Regression Model: " + Arrays.toString(linearRegressionMdl)); } } diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/CacheBasedDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/CacheBasedDatasetExample.java index adae6615a9065..3b82c4eaee1d7 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/CacheBasedDatasetExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/CacheBasedDatasetExample.java @@ -32,41 +32,41 @@ */ public class CacheBasedDatasetExample { /** Run example. */ - public static void main(String[] args) { + public static void main(String[] args) throws Exception { try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { System.out.println(">>> D-Learn Cache Dataset example started."); IgniteCache persons = createCache(ignite); // Creates a cache based simple dataset containing features and providing standard dataset API. - SimpleDataset dataset = DatasetFactory.createSimpleDataset( + try (SimpleDataset dataset = DatasetFactory.createSimpleDataset( ignite, persons, (k, v) -> new double[]{ v.getAge(), v.getSalary() }, 2 - ); + )) { + // Calculation of the mean value. This calculation will be performed in map-reduce manner. + double[] mean = dataset.mean(); + System.out.println("Mean \n\t" + Arrays.toString(mean)); - // Calculation of the mean value. This calculation will be performed in map-reduce manner. - double[] mean = dataset.mean(); - System.out.println("Mean \n\t" + Arrays.toString(mean)); + // Calculation of the standard deviation. This calculation will be performed in map-reduce manner. + double[] std = dataset.std(); + System.out.println("Standard deviation \n\t" + Arrays.toString(std)); - // Calculation of the standard deviation. This calculation will be performed in map-reduce manner. - double[] std = dataset.std(); - System.out.println("Standard deviation \n\t" + Arrays.toString(std)); + // Calculation of the covariance matrix. This calculation will be performed in map-reduce manner. + double[][] cov = dataset.cov(); + System.out.println("Covariance matrix "); + for (double[] row : cov) + System.out.println("\t" + Arrays.toString(row)); - // Calculation of the covariance matrix. This calculation will be performed in map-reduce manner. - double[][] cov = dataset.cov(); - System.out.println("Covariance matrix "); - for (double[] row : cov) - System.out.println("\t" + Arrays.toString(row)); + // Calculation of the correlation matrix. This calculation will be performed in map-reduce manner. + double[][] corr = dataset.corr(); + System.out.println("Correlation matrix "); + for (double[] row : corr) + System.out.println("\t" + Arrays.toString(row)); - // Calculation of the correlation matrix. This calculation will be performed in map-reduce manner. - double[][] corr = dataset.corr(); - System.out.println("Correlation matrix "); - for (double[] row : corr) - System.out.println("\t" + Arrays.toString(row)); - - System.out.println(">>> D-Learn Cache Dataset example completed."); + System.out.println(">>> D-Learn Cache Dataset example completed."); + } } } diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalDatasetExample.java index 4c563c2b2cc78..9888fab97d9ad 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalDatasetExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalDatasetExample.java @@ -31,41 +31,41 @@ */ public class LocalDatasetExample { /** Run example. */ - public static void main(String[] args) { + public static void main(String[] args) throws Exception { try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { System.out.println(">>> D-Learn Cache Dataset example started."); Map persons = createCache(ignite); // Creates a local simple dataset containing features and providing standard dataset API. - SimpleDataset dataset = DatasetFactory.createSimpleDataset( + try (SimpleDataset dataset = DatasetFactory.createSimpleDataset( persons, 2, (k, v) -> new double[]{ v.getAge(), v.getSalary() }, 2 - ); + )) { + // Calculation of the mean value. This calculation will be performed in map-reduce manner. + double[] mean = dataset.mean(); + System.out.println("Mean \n\t" + Arrays.toString(mean)); - // Calculation of the mean value. This calculation will be performed in map-reduce manner. - double[] mean = dataset.mean(); - System.out.println("Mean \n\t" + Arrays.toString(mean)); + // Calculation of the standard deviation. This calculation will be performed in map-reduce manner. + double[] std = dataset.std(); + System.out.println("Standard deviation \n\t" + Arrays.toString(std)); - // Calculation of the standard deviation. This calculation will be performed in map-reduce manner. - double[] std = dataset.std(); - System.out.println("Standard deviation \n\t" + Arrays.toString(std)); + // Calculation of the covariance matrix. This calculation will be performed in map-reduce manner. + double[][] cov = dataset.cov(); + System.out.println("Covariance matrix "); + for (double[] row : cov) + System.out.println("\t" + Arrays.toString(row)); - // Calculation of the covariance matrix. This calculation will be performed in map-reduce manner. - double[][] cov = dataset.cov(); - System.out.println("Covariance matrix "); - for (double[] row : cov) - System.out.println("\t" + Arrays.toString(row)); + // Calculation of the correlation matrix. This calculation will be performed in map-reduce manner. + double[][] corr = dataset.corr(); + System.out.println("Correlation matrix "); + for (double[] row : corr) + System.out.println("\t" + Arrays.toString(row)); - // Calculation of the correlation matrix. This calculation will be performed in map-reduce manner. - double[][] corr = dataset.corr(); - System.out.println("Correlation matrix "); - for (double[] row : corr) - System.out.println("\t" + Arrays.toString(row)); - - System.out.println(">>> D-Learn Cache Dataset example completed."); + System.out.println(">>> D-Learn Cache Dataset example completed."); + } } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java index cd119a133d261..4482af7ddfabf 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/UpstreamCursorAdapter.java @@ -23,7 +23,7 @@ import org.apache.ignite.ml.dataset.UpstreamEntry; /** - * Cursor adapter used to transform {@link Cache.Entry} received from Ignite Cache query cursor into DLC-specific + * Cursor adapter used to transform {@code Cache.Entry} received from Ignite Cache query cursor into DLC-specific * {@link UpstreamEntry}. * * @param Type of an upstream value key. diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java index d5d6d9484f3a7..88e7d4e020209 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java @@ -18,6 +18,7 @@ package org.apache.ignite.ml; import org.apache.ignite.ml.clustering.ClusteringTestSuite; +import org.apache.ignite.ml.dataset.DatasetTestSuite; import org.apache.ignite.ml.knn.KNNTestSuite; import org.apache.ignite.ml.math.MathImplMainTestSuite; import org.apache.ignite.ml.nn.MLPTestSuite; @@ -41,7 +42,9 @@ LocalModelsTest.class, MLPTestSuite.class, TrainersGroupTestSuite.class, - OptimizationTestSuite.class + OptimizationTestSuite.class, + DatasetTestSuite.class + }) public class IgniteMLTestSuite { // No-op. diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/DatasetTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/DatasetTestSuite.java new file mode 100644 index 0000000000000..eab104a2d4b15 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/DatasetTestSuite.java @@ -0,0 +1,45 @@ +/* + * 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.ml.dataset; + +import org.apache.ignite.ml.dataset.api.DatasetWrapperTest; +import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilderTest; +import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetTest; +import org.apache.ignite.ml.dataset.impl.cache.util.ComputeUtilsTest; +import org.apache.ignite.ml.dataset.impl.cache.util.DatasetAffinityFunctionWrapperTest; +import org.apache.ignite.ml.dataset.impl.cache.util.PartitionDataStorageTest; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilderTest; +import org.junit.runner.RunWith; +import org.junit.runners.Suite; + +/** + * Test suite for all tests located in org.apache.ignite.ml.dataset.* package. + */ +@RunWith(Suite.class) +@Suite.SuiteClasses({ + DatasetWrapperTest.class, + ComputeUtilsTest.class, + DatasetAffinityFunctionWrapperTest.class, + PartitionDataStorageTest.class, + CacheBasedDatasetBuilderTest.class, + CacheBasedDatasetTest.class, + LocalDatasetBuilderTest.class +}) +public class DatasetTestSuite { + // No-op. +} From bfbf7acc9eefb0903136be3801ac78549f04e69c Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Thu, 1 Feb 2018 16:32:40 +0300 Subject: [PATCH 32/38] IGNITE-7438 Initial version of the LSQR algorithm implementation. --- .../org/apache/ignite/ml/DatasetTrainer.java | 42 +++ .../impl/local/LocalDatasetBuilder.java | 7 +- .../ml/math/isolve/IterativeSolverResult.java | 64 ++++ .../LinSysPartitionDataBuilderOnHeap.java | 85 +++++ .../isolve/LinSysPartitionDataOnHeap.java | 75 +++++ .../ml/math/isolve/lsqr/AbstractLSQR.java | 292 ++++++++++++++++++ .../ml/math/isolve/lsqr/LSQROnHeap.java | 98 ++++++ .../isolve/lsqr/LSQRPartitionContext.java | 41 +++ .../ml/math/isolve/lsqr/LSQRResult.java | 140 +++++++++ .../ml/math/isolve/lsqr/package-info.java | 22 ++ .../ignite/ml/math/isolve/package-info.java | 22 ++ .../linear/LinearRegressionLSQRTrainer.java | 70 +++++ .../ml/math/MathImplLocalTestSuite.java | 4 +- .../ml/math/isolve/lsqr/LSQROnHeapTest.java | 109 +++++++ .../ml/regressions/RegressionsTestSuite.java | 4 +- .../LinearRegressionLSQRTrainerTest.java | 124 ++++++++ 16 files changed, 1195 insertions(+), 4 deletions(-) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/DatasetTrainer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/IterativeSolverResult.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataBuilderOnHeap.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataOnHeap.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRPartitionContext.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRResult.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainerTest.java diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/DatasetTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/DatasetTrainer.java new file mode 100644 index 0000000000000..d1a68d5b3706c --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/DatasetTrainer.java @@ -0,0 +1,42 @@ +/* + * 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.ml; + +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +/** + * Interface for Trainers. Trainer is just a function which produces model from the data. + * + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @param Type of a produced model. + */ +public interface DatasetTrainer { + /** + * Trains model based on the specified data. + * + * @param datasetBuilder Dataset builder. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + * @param cols Number of columns. + * @return Model. + */ + public M fit(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor, int cols); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java index 42c3cccdffcff..f4f4d63e061c3 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java @@ -60,13 +60,14 @@ public LocalDatasetBuilder(Map upstreamMap, int partitions) { List ctxList = new ArrayList<>(); List dataList = new ArrayList<>(); - int partSize = upstreamMap.size() / partitions; + int partSize = Math.max(1, upstreamMap.size() / partitions); Iterator firstKeysIter = upstreamMap.keySet().iterator(); Iterator secondKeysIter = upstreamMap.keySet().iterator(); + int ptr = 0; for (int part = 0; part < partitions; part++) { - int cnt = Math.min((part + 1) * partSize, upstreamMap.size()) - part * partSize; + int cnt = Math.max(partSize, upstreamMap.size() - ptr); C ctx = partCtxBuilder.build( new IteratorWindow<>(firstKeysIter, k -> new UpstreamEntry<>(k, upstreamMap.get(k)), cnt), @@ -81,6 +82,8 @@ public LocalDatasetBuilder(Map upstreamMap, int partitions) { ctxList.add(ctx); dataList.add(data); + + ptr += cnt; } return new LocalDataset<>(ctxList, dataList); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/IterativeSolverResult.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/IterativeSolverResult.java new file mode 100644 index 0000000000000..fe39ad7f7fec1 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/IterativeSolverResult.java @@ -0,0 +1,64 @@ +/* + * 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.ml.math.isolve; + +import java.io.Serializable; +import java.util.Arrays; + +/** + * Base class for iterative solver results. + */ +public class IterativeSolverResult implements Serializable { + /** */ + private static final long serialVersionUID = 8084061028708491097L; + + /** The final solution. */ + private final double[] x; + + /** Iteration number upon termination. */ + private final int iterations; + + /** + * Constructs a new instance of iterative solver result. + * + * @param x The final solution. + * @param iterations Iteration number upon termination. + */ + public IterativeSolverResult(double[] x, int iterations) { + this.x = x; + this.iterations = iterations; + } + + /** */ + public double[] getX() { + return x; + } + + /** */ + public int getIterations() { + return iterations; + } + + /** */ + @Override public String toString() { + return "IterativeSolverResult{" + + "x=" + Arrays.toString(x) + + ", iterations=" + iterations + + '}'; + } +} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataBuilderOnHeap.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataBuilderOnHeap.java new file mode 100644 index 0000000000000..1c2e2cf9cc2bb --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataBuilderOnHeap.java @@ -0,0 +1,85 @@ +/* + * 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.ml.math.isolve; + +import java.io.Serializable; +import java.util.Iterator; +import org.apache.ignite.ml.dataset.PartitionDataBuilder; +import org.apache.ignite.ml.dataset.UpstreamEntry; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +/** + * Linear system partition data builder that builds {@link LinSysPartitionDataOnHeap}. + * + * @param Type of a key in upstream data. + * @param Type of a value in upstream data. + * @param Type of a partition context. + */ +public class LinSysPartitionDataBuilderOnHeap + implements PartitionDataBuilder { + /** */ + private static final long serialVersionUID = -7820760153954269227L; + + /** Extractor of X matrix row. */ + private final IgniteBiFunction xExtractor; + + /** Extractor of Y vector value. */ + private final IgniteBiFunction yExtractor; + + /** Number of columns. */ + private final int cols; + + /** + * Constructs a new instance of linear system partition data builder. + * + * @param xExtractor Extractor of X matrix row. + * @param yExtractor Extractor of Y vector value. + * @param cols Number of columns. + */ + public LinSysPartitionDataBuilderOnHeap(IgniteBiFunction xExtractor, + IgniteBiFunction yExtractor, int cols) { + this.xExtractor = xExtractor; + this.yExtractor = yExtractor; + this.cols = cols; + } + + /** {@inheritDoc} */ + @Override public LinSysPartitionDataOnHeap build(Iterator> upstreamData, long upstreamDataSize, + C ctx) { + // Prepares the matrix of features in flat column-major format. + double[] x = new double[Math.toIntExact(upstreamDataSize * cols)]; + double[] y = new double[Math.toIntExact(upstreamDataSize)]; + + int ptr = 0; + while (upstreamData.hasNext()) { + UpstreamEntry entry = upstreamData.next(); + double[] row = xExtractor.apply(entry.getKey(), entry.getValue()); + + assert row.length == cols : "X extractor must return exactly " + cols + " columns"; + + for (int i = 0; i < cols; i++) + x[Math.toIntExact(i * upstreamDataSize) + ptr] = row[i]; + + y[ptr] = yExtractor.apply(entry.getKey(), entry.getValue()); + + ptr++; + } + + return new LinSysPartitionDataOnHeap(x, Math.toIntExact(upstreamDataSize), cols, y); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataOnHeap.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataOnHeap.java new file mode 100644 index 0000000000000..e0b8f46049d19 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataOnHeap.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.ignite.ml.math.isolve; + +/** + * On Heap partition data that keeps part of a linear system. + */ +public class LinSysPartitionDataOnHeap implements AutoCloseable { + /** Part of X matrix. */ + private final double[] x; + + /** Number of rows. */ + private final int rows; + + /** Number of columns. */ + private final int cols; + + /** Part of Y vector. */ + private final double[] y; + + /** + * Constructs a new instance of linear system partition data. + * + * @param x Part of X matrix. + * @param rows Number of rows. + * @param cols Number of columns. + * @param y Part of Y vector. + */ + public LinSysPartitionDataOnHeap(double[] x, int rows, int cols, double[] y) { + this.x = x; + this.rows = rows; + this.cols = cols; + this.y = y; + } + + /** */ + public double[] getX() { + return x; + } + + /** */ + public int getRows() { + return rows; + } + + /** */ + public int getCols() { + return cols; + } + + /** */ + public double[] getY() { + return y; + } + + /** {@inheritDoc} */ + @Override public void close() { + // Do nothing, GC will clean up. + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java new file mode 100644 index 0000000000000..070f9c08c356b --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java @@ -0,0 +1,292 @@ +/* + * 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.ml.math.isolve.lsqr; + +import com.github.fommil.netlib.BLAS; +import java.util.Arrays; + +/** + * Basic implementation of the LSQR algorithm without assumptions about dataset storage format or data processing + * device. + */ +public abstract class AbstractLSQR { + /** + * The smallest representable positive number such that 1.0 + eps != 1.0. + */ + private static final double eps = Double.longBitsToDouble(Double.doubleToLongBits(1.0) | 1) - 1.0; + + /** */ + private static BLAS blas = BLAS.getInstance(); + + /** + * Solves given Sparse Linear Systems. + * + * @param damp damping coefficient + * @param atol stopping tolerances, if both (atol and btol) are 1.0e-9 (say), the final residual norm should be + * accurate to about 9 digits. (The final x will usually have fewer correct digits, depending on cond(A) and the + * size of damp.) + * @param btol stopping tolerances, if both (atol and btol) are 1.0e-9 (say), the final residual norm should be + * accurate to about 9 digits. (The final x will usually have fewer correct digits, depending on cond(A) and the + * size of damp.) + * @param conlim another stopping tolerance, lsqr terminates if an estimate of cond(A) exceeds conlim. For + * compatible systems Ax = b, conlim could be as large as 1.0e+12 (say). For least-squares problems, conlim should + * be less than 1.0e+8. Maximum precision can be obtained by setting atol = btol = conlim = zero, but the number of + * iterations may then be excessive. + * @param iterLim explicit limitation on number of iterations (for safety) + * @param calcVar whether to estimate diagonals of (A'A + damp^2*I)^{-1} + * @param x0 initial value of x + * @return solver result + */ + public LSQRResult solve(double damp, double atol, double btol, double conlim, double iterLim, boolean calcVar, double[] x0) { + int n = getColumns(); + if (iterLim < 0) + iterLim = 2 * n; + double[] var = new double[n]; + int itn = 0; + int istop = 0; + double ctol = 0; + if (conlim > 0) + ctol = 1 / conlim; + double anorm = 0; + double acond = 0; + double dampsq = Math.pow(damp, 2.0); + double ddnorm = 0; + double res2 = 0; + double xnorm = 0; + double xxnorm = 0; + double z = 0; + double cs2 = -1; + double sn2 = 0; + + // Set up the first vectors u and v for the bidiagonalization. + // These satisfy beta*u = b - A*x, alfa*v = A'*u. + double bnorm = bnorm(); + double[] x; + double beta; + if (x0 == null) { + x = new double[n]; + beta = bnorm; + } + else { + x = x0; + beta = beta(x, -1.0, 1.0); + } + double[] v = new double[n]; + double alfa; + if (beta > 0) { + v = iter(beta, v); + alfa = blas.dnrm2(v.length, v, 1); + } + else { + System.arraycopy(x, 0, v, 0, v.length); + alfa = 0; + } + + if (alfa > 0) + blas.dscal(v.length, 1 / alfa, v, 1); + double[] w = Arrays.copyOf(v, v.length); + + double rhobar = alfa; + double phibar = beta; + double rnorm = beta; + double r1norm = rnorm; + double r2norm = rnorm; + double arnorm = alfa * beta; + double[] dk = new double[w.length]; + if (arnorm == 0) + return new LSQRResult(x, itn, istop, r1norm, r2norm, anorm, acond, arnorm, xnorm, var); + + // Main iteration loop. + while (itn < iterLim) { + itn = itn + 1; + + // Perform the next step of the bidiagonalization to obtain the + // next beta, u, alfa, v. These satisfy the relations + // beta*u = A*v - alfa*u, + // alfa*v = A'*u - beta*v. + beta = beta(v, 1.0, -alfa); + if (beta > 0) { + anorm = Math.sqrt(Math.pow(anorm, 2) + Math.pow(alfa, 2) + Math.pow(beta, 2) + Math.pow(damp, 2)); + blas.dscal(v.length, -beta, v, 1); + iter(beta, v); + //v = dataset.iter(beta, n); + alfa = blas.dnrm2(v.length, v, 1); + if (alfa > 0) + blas.dscal(v.length, 1 / alfa, v, 1); + } + + // Use a plane rotation to eliminate the damping parameter. + // This alters the diagonal (rhobar) of the lower-bidiagonal matrix. + double rhobar1 = Math.sqrt(Math.pow(rhobar, 2) + Math.pow(damp, 2)); + double cs1 = rhobar / rhobar1; + double sn1 = damp / rhobar1; + double psi = sn1 * phibar; + phibar = cs1 * phibar; + + // Use a plane rotation to eliminate the subdiagonal element (beta) + // of the lower-bidiagonal matrix, giving an upper-bidiagonal matrix. + double[] symOrtho = symOrtho(rhobar1, beta); + double cs = symOrtho[0]; + double sn = symOrtho[1]; + double rho = symOrtho[2]; + + double theta = sn * alfa; + rhobar = -cs * alfa; + double phi = cs * phibar; + phibar = sn * phibar; + double tau = sn * phi; + + double t1 = phi / rho; + double t2 = -theta / rho; + blas.dcopy(w.length, w, 1, dk, 1); + blas.dscal(dk.length, 1 / rho, dk, 1); + + // x = x + t1*w + blas.daxpy(w.length, t1, w, 1, x, 1); + // w = v + t2*w + blas.dscal(w.length, t2, w, 1); + blas.daxpy(w.length, 1, v, 1, w, 1); + ddnorm = ddnorm + Math.pow(blas.dnrm2(dk.length, dk, 1), 2); + + if (calcVar) + blas.daxpy(var.length, 1.0, pow(dk, 2), 1, var, 1); + + // Use a plane rotation on the right to eliminate the + // super-diagonal element (theta) of the upper-bidiagonal matrix. + // Then use the result to estimate norm(x). + double delta = sn2 * rho; + double gambar = -cs2 * rho; + double rhs = phi - delta * z; + double zbar = rhs / gambar; + xnorm = Math.sqrt(xxnorm + Math.pow(zbar, 2)); + double gamma = Math.sqrt(Math.pow(gambar, 2) + Math.pow(theta, 2)); + cs2 = gambar / gamma; + sn2 = theta / gamma; + z = rhs / gamma; + xxnorm = xxnorm + Math.pow(z, 2); + + // Test for convergence. + // First, estimate the condition of the matrix Abar, + // and the norms of rbar and Abar'rbar. + acond = anorm * Math.sqrt(ddnorm); + double res1 = Math.pow(phibar, 2); + res2 = res2 + Math.pow(psi, 2); + rnorm = Math.sqrt(res1 + res2); + arnorm = alfa * Math.abs(tau); + + // Distinguish between + // r1norm = ||b - Ax|| and + // r2norm = rnorm in current code + // = sqrt(r1norm^2 + damp^2*||x||^2). + // Estimate r1norm from + // r1norm = sqrt(r2norm^2 - damp^2*||x||^2). + // Although there is cancellation, it might be accurate enough. + double r1sq = Math.pow(rnorm, 2) - dampsq * xxnorm; + r1norm = Math.sqrt(Math.abs(r1sq)); + if (r1sq < 0) + r1norm = -r1norm; + r2norm = rnorm; + + // Now use these norms to estimate certain other quantities, + // some of which will be small near a solution. + double test1 = rnorm / bnorm; + double test2 = arnorm / (anorm * rnorm + eps); + double test3 = 1 / (acond + eps); + t1 = test1 / (1 + anorm * xnorm / bnorm); + double rtol = btol + atol * anorm * xnorm / bnorm; + + // The following tests guard against extremely small values of + // atol, btol or ctol. (The user may have set any or all of + // the parameters atol, btol, conlim to 0.) + // The effect is equivalent to the normal tests using + // atol = eps, btol = eps, conlim = 1/eps. + if (itn >= iterLim) + istop = 7; + if (1 + test3 <= 1) + istop = 6; + if (1 + test2 <= 1) + istop = 5; + if (1 + t1 <= 1) + istop = 4; + + // Allow for tolerances set by the user. + if (test3 <= ctol) + istop = 3; + if (test2 <= atol) + istop = 2; + if (test1 <= rtol) + istop = 1; + + if (istop != 0) + break; + } + return new LSQRResult(x, itn, istop, r1norm, r2norm, anorm, acond, arnorm, xnorm, var); + } + + /** */ + protected abstract double bnorm(); + + /** */ + protected abstract double beta(double[] x, double alfa, double beta); + + /** */ + protected abstract double[] iter(double bnorm, double[] target); + + /** */ + protected abstract int getColumns(); + + /** */ + private static double[] symOrtho(double a, double b) { + if (b == 0) + return new double[]{ Math.signum(a), 0, Math.abs(a) }; + else if (a == 0) + return new double[]{ 0, Math.signum(b), Math.abs(b) }; + else { + double c, s, r; + if (Math.abs(b) > Math.abs(a)) { + double tau = a / b; + s = Math.signum(b) / Math.sqrt(1 + tau * tau); + c = s * tau; + r = b / s; + } + else { + double tau = b / a; + c = Math.signum(a) / Math.sqrt(1 + tau * tau); + s = c * tau; + r = a / c; + } + return new double[]{ c, s, r }; + } + } + + /** + * Raises all elements of the specified vector {@code a} to the power of the specified {@code pow}. Be aware that + * it's "in place" operation. + * + * @param a Vector or matrix of doubles. + * @param pow Power. + * @return Matrix with elements raised to the specified power. + */ + private static double[] pow(double[] a, double pow) { + double[] res = new double[a.length]; + for (int i = 0; i < res.length; i++) + res[i] = Math.pow(a[i], pow); + return res; + } + +} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java new file mode 100644 index 0000000000000..75688022b274a --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java @@ -0,0 +1,98 @@ +/* + * 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.ml.math.isolve.lsqr; + +import com.github.fommil.netlib.BLAS; +import java.util.Arrays; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.dataset.PartitionDataBuilder; +import org.apache.ignite.ml.math.isolve.LinSysPartitionDataOnHeap; + +/** + * Distributed implementation of LSQR algorithm based on. + */ +public class LSQROnHeap extends AbstractLSQR implements AutoCloseable { + + private final Dataset dataset; + + /** + * Constructs a new instance of OnHeap LSQR algorithm implementation. + * + * @param datasetBuilder Dataset builder. + * @param partDataBuilder Partition data builder. + */ + public LSQROnHeap(DatasetBuilder datasetBuilder, + PartitionDataBuilder partDataBuilder) { + this.dataset = datasetBuilder.build( + (upstream, upstreamSize) -> new LSQRPartitionContext(), + partDataBuilder + ); + } + + /** {@inheritDoc} */ + @Override protected double bnorm() { + double res = dataset.computeWithCtx((ctx, data) -> { + ctx.setU(Arrays.copyOf(data.getY(), data.getY().length)); + return BLAS.getInstance().dnrm2(data.getY().length, data.getY(), 1); + }, (a, b) -> a == null ? b : Math.sqrt(a * a + b * b)); + return res; + } + + /** {@inheritDoc} */ + @Override protected double beta(double[] x, double alfa, double beta) { + double bb = dataset.computeWithCtx((ctx, data) -> { + BLAS.getInstance().dgemv("N", data.getRows(), data.getCols(), alfa, data.getX(), data.getRows(), x, 1, beta, ctx.getU(), 1); + return BLAS.getInstance().dnrm2(ctx.getU().length, ctx.getU(), 1); + }, (a, b) -> a == null ? b : Math.sqrt(a * a + b * b)); + return bb; + } + + /** {@inheritDoc} */ + @Override protected double[] iter(double bnorm, double[] target) { + double[] res = dataset.computeWithCtx((ctx, data) -> { + BLAS.getInstance().dscal(ctx.getU().length, 1 / bnorm, ctx.getU(), 1); + double[] v = new double[data.getCols()]; + BLAS.getInstance().dgemv("T", data.getRows(), data.getCols(), 1.0, data.getX(), data.getRows(), ctx.getU(), 1, 0, v, 1); + return v; + }, (a, b) -> { + if (a == null) + return b; + else { + BLAS.getInstance().daxpy(a.length, 1.0, a, 1, b, 1); + return b; + } + }); + BLAS.getInstance().daxpy(res.length, 1.0, res, 1, target, 1); + return target; + } + + /** + * Returns number of columns in dataset. + * + * @return number of columns + */ + @Override protected int getColumns() { + return dataset.compute(LinSysPartitionDataOnHeap::getCols, (a, b) -> a == null ? b : a); + } + + /** {@inheritDoc} */ + @Override public void close() throws Exception { + dataset.close(); + } +} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRPartitionContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRPartitionContext.java new file mode 100644 index 0000000000000..5769cfb5a5279 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRPartitionContext.java @@ -0,0 +1,41 @@ +/* + * 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.ml.math.isolve.lsqr; + +import java.io.Serializable; + +/** + * Partition context of the LSQR algorithm. + */ +public class LSQRPartitionContext implements Serializable { + /** */ + private static final long serialVersionUID = -8159608186899430315L; + + /** Part of V vector. */ + private double[] u; + + /** */ + public double[] getU() { + return u; + } + + /** */ + public void setU(double[] u) { + this.u = u; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRResult.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRResult.java new file mode 100644 index 0000000000000..d320ea677227b --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRResult.java @@ -0,0 +1,140 @@ +/* + * 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.ml.math.isolve.lsqr; + +import java.util.Arrays; +import org.apache.ignite.ml.math.isolve.IterativeSolverResult; + +/** + * LSQR iterative solver result. + */ +public class LSQRResult extends IterativeSolverResult { + /** */ + private static final long serialVersionUID = -8866269808589635947L; + + /** + * Gives the reason for termination. 1 means x is an approximate solution to Ax = b. 2 means x approximately solves + * the least-squares problem. + */ + private final int isstop; + + /** + * Represents norm(r), where r = b - Ax. + */ + private final double r1norn; + + /** + * Represents sqrt( norm(r)^2 + damp^2 * norm(x)^2 ). Equal to r1norm if damp == 0. + */ + private final double r2norm; + + /** + * Estimate of Frobenius norm of Abar = [[A]; [damp*I]]. + */ + private final double anorm; + + /** + * Estimate of cond(Abar). + */ + private final double acond; + + /** + * Estimate of norm(A'*r - damp^2*x). + */ + private final double arnorm; + + /** + * Represents norm(x). + */ + private final double xnorm; + + /** + * If calc_var is True, estimates all diagonals of (A'A)^{-1} (if damp == 0) or more generally + * (A'A + damp^2*I)^{-1}. This is well defined if A has full column rank or damp > 0. (Not sure what var means if + * rank(A) < n and damp = 0.) + */ + private final double[] var; + + /** */ + public LSQRResult(double[] x, int iterations, int isstop, double r1norn, double r2norm, double anorm, double acond, + double arnorm, double xnorm, double[] var) { + super(x, iterations); + this.isstop = isstop; + this.r1norn = r1norn; + this.r2norm = r2norm; + this.anorm = anorm; + this.acond = acond; + this.arnorm = arnorm; + this.xnorm = xnorm; + this.var = var; + } + + /** */ + public int getIsstop() { + return isstop; + } + + /** */ + public double getR1norn() { + return r1norn; + } + + /** */ + public double getR2norm() { + return r2norm; + } + + /** */ + public double getAnorm() { + return anorm; + } + + /** */ + public double getAcond() { + return acond; + } + + /** */ + public double getArnorm() { + return arnorm; + } + + /** */ + public double getXnorm() { + return xnorm; + } + + /** */ + public double[] getVar() { + return var; + } + + /** */ + @Override public String toString() { + return "LSQRResult{" + + "isstop=" + isstop + + ", r1norn=" + r1norn + + ", r2norm=" + r2norm + + ", anorm=" + anorm + + ", acond=" + acond + + ", arnorm=" + arnorm + + ", xnorm=" + xnorm + + ", var=" + Arrays.toString(var) + + '}'; + } +} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/package-info.java new file mode 100644 index 0000000000000..a667eb7f811a7 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Contains LSQR algorithm implementation. + */ +package org.apache.ignite.ml.math.isolve.lsqr; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/package-info.java new file mode 100644 index 0000000000000..5e0155f4bb016 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Contains iterative algorithms for solving linear systems. + */ +package org.apache.ignite.ml.math.isolve; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java new file mode 100644 index 0000000000000..d7d587e1f38ed --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java @@ -0,0 +1,70 @@ +/* + * 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.ml.regressions.linear; + +import java.util.Arrays; +import org.apache.ignite.ml.DatasetTrainer; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.math.Vector; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; +import org.apache.ignite.ml.math.isolve.LinSysPartitionDataBuilderOnHeap; +import org.apache.ignite.ml.math.isolve.lsqr.AbstractLSQR; +import org.apache.ignite.ml.math.isolve.lsqr.LSQROnHeap; +import org.apache.ignite.ml.math.isolve.lsqr.LSQRResult; + +/** + * Trainer of the linear regression model based on LSQR algorithm. + * + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * + * @see AbstractLSQR + */ +public class LinearRegressionLSQRTrainer implements DatasetTrainer { + /** {@inheritDoc} */ + @Override public LinearRegressionModel fit(DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor, int cols) { + + LSQRResult res; + + try (LSQROnHeap lsqr = new LSQROnHeap<>( + datasetBuilder, + new LinSysPartitionDataBuilderOnHeap<>( + (k, v) -> { + double[] row = Arrays.copyOf(featureExtractor.apply(k, v), cols + 1); + + row[cols] = 1.0; + + return row; + }, + lbExtractor, + cols + 1 + ) + )) { + res = lsqr.solve(0, 1e-12, 1e-12, 1e8, -1, false, null); + } + catch (Exception e) { + throw new RuntimeException(e); + } + + Vector weights = new DenseLocalOnHeapVector(Arrays.copyOfRange(res.getX(), 0, cols)); + + return new LinearRegressionModel(weights, res.getX()[cols]); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/math/MathImplLocalTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/math/MathImplLocalTestSuite.java index bb41239367666..926d8726efcee 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/math/MathImplLocalTestSuite.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/math/MathImplLocalTestSuite.java @@ -61,6 +61,7 @@ import org.apache.ignite.ml.math.impls.vector.VectorNormTest; import org.apache.ignite.ml.math.impls.vector.VectorToMatrixTest; import org.apache.ignite.ml.math.impls.vector.VectorViewTest; +import org.apache.ignite.ml.math.isolve.lsqr.LSQROnHeapTest; import org.junit.runner.RunWith; import org.junit.runners.Suite; @@ -120,7 +121,8 @@ QRDecompositionTest.class, SingularValueDecompositionTest.class, QRDSolverTest.class, - DistanceTest.class + DistanceTest.class, + LSQROnHeapTest.class }) public class MathImplLocalTestSuite { // No-op. diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java new file mode 100644 index 0000000000000..5c080886f87e5 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java @@ -0,0 +1,109 @@ +/* + * 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.ml.math.isolve.lsqr; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; +import org.apache.ignite.ml.math.isolve.LinSysPartitionDataBuilderOnHeap; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.junit.Assert.assertArrayEquals; + +/** + * Tests for {@link LSQROnHeap}. + */ +@RunWith(Parameterized.class) +public class LSQROnHeapTest { + /** Parameters. */ + @Parameterized.Parameters(name = "Data divided on {0} partitions") + public static Iterable data() { + return Arrays.asList( + new Integer[] {1}, + new Integer[] {2}, + new Integer[] {3}, + new Integer[] {5}, + new Integer[] {7}, + new Integer[] {100}, + new Integer[] {1000} + ); + } + + /** Number of partitions. */ + @Parameterized.Parameter + public int parts; + + /** Tests solving simple linear system. */ + @Test + public void testSolveLinearSystem() { + Map data = new HashMap<>(); + data.put(0, new double[]{3, 2, -1, 1}); + data.put(1, new double[]{2, -2, 4, -2}); + data.put(2, new double[]{-1, 0.5, -1, 0}); + + DatasetBuilder datasetBuilder = new LocalDatasetBuilder<>(data, parts); + + LSQROnHeap lsqr = new LSQROnHeap<>( + datasetBuilder, + new LinSysPartitionDataBuilderOnHeap<>( + (k, v) -> Arrays.copyOf(v, v.length - 1), + (k, v) -> v[3], + 3 + ) + ); + + LSQRResult res = lsqr.solve(0, 1e-12, 1e-12, 1e8, -1, false, null); + + assertArrayEquals(new double[]{1, -2, -2}, res.getX(), 1e-6); + } + + /** Tests solving least squares problem. */ + @Test + public void testSolveLeastSquares() throws Exception { + Map data = new HashMap<>(); + data.put(0, new double[] {-1.0915526, 1.81983527, -0.91409478, 0.70890712, -24.55724107}); + data.put(1, new double[] {-0.61072904, 0.37545517, 0.21705352, 0.09516495, -26.57226867}); + data.put(2, new double[] {0.05485406, 0.88219898, -0.80584547, 0.94668307, 61.80919728}); + data.put(3, new double[] {-0.24835094, -0.34000053, -1.69984651, -1.45902635, -161.65525991}); + data.put(4, new double[] {0.63675392, 0.31675535, 0.38837437, -1.1221971, -14.46432611}); + data.put(5, new double[] {0.14194017, 2.18158997, -0.28397346, -0.62090588, -3.2122197}); + data.put(6, new double[] {-0.53487507, 1.4454797, 0.21570443, -0.54161422, -46.5469012}); + data.put(7, new double[] {-1.58812173, -0.73216803, -2.15670676, -1.03195988, -247.23559889}); + data.put(8, new double[] {0.20702671, 0.92864654, 0.32721202, -0.09047503, 31.61484949}); + data.put(9, new double[] {-0.37890345, -0.04846179, -0.84122753, -1.14667474, -124.92598583}); + + DatasetBuilder datasetBuilder = new LocalDatasetBuilder<>(data, 1); + + try (LSQROnHeap lsqr = new LSQROnHeap<>( + datasetBuilder, + new LinSysPartitionDataBuilderOnHeap<>( + (k, v) -> Arrays.copyOf(v, v.length - 1), + (k, v) -> v[4], + 4 + ) + )) { + LSQRResult res = lsqr.solve(0, 1e-12, 1e-12, 1e8, -1, false, null); + + assertArrayEquals(new double[]{72.26948107, 15.95144674, 24.07403921, 66.73038781}, res.getX(), 1e-6); + } + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java index 5c79c8fa8239a..82b3a1b24df63 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java @@ -21,6 +21,7 @@ import org.apache.ignite.ml.regressions.linear.BlockDistributedLinearRegressionSGDTrainerTest; import org.apache.ignite.ml.regressions.linear.DistributedLinearRegressionQRTrainerTest; import org.apache.ignite.ml.regressions.linear.DistributedLinearRegressionSGDTrainerTest; +import org.apache.ignite.ml.regressions.linear.LinearRegressionLSQRTrainerTest; import org.apache.ignite.ml.regressions.linear.LinearRegressionModelTest; import org.apache.ignite.ml.regressions.linear.LocalLinearRegressionQRTrainerTest; import org.apache.ignite.ml.regressions.linear.LocalLinearRegressionSGDTrainerTest; @@ -38,7 +39,8 @@ DistributedLinearRegressionQRTrainerTest.class, DistributedLinearRegressionSGDTrainerTest.class, BlockDistributedLinearRegressionQRTrainerTest.class, - BlockDistributedLinearRegressionSGDTrainerTest.class + BlockDistributedLinearRegressionSGDTrainerTest.class, + LinearRegressionLSQRTrainerTest.class }) public class RegressionsTestSuite { // No-op. diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainerTest.java new file mode 100644 index 0000000000000..3bb3ee701186e --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainerTest.java @@ -0,0 +1,124 @@ +/* + * 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.ml.regressions.linear; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Random; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link LinearRegressionLSQRTrainer}. + */ +@RunWith(Parameterized.class) +public class LinearRegressionLSQRTrainerTest { + /** Parameters. */ + @Parameterized.Parameters(name = "Data divided on {0} partitions") + public static Iterable data() { + return Arrays.asList( + new Integer[] {1}, + new Integer[] {2}, + new Integer[] {3}, + new Integer[] {5}, + new Integer[] {7}, + new Integer[] {100}, + new Integer[] {1000} + ); + } + + /** Number of partitions. */ + @Parameterized.Parameter + public int parts; + + /** + * Tests {@code fit()} method on a simple small dataset. + */ + @Test + public void testSmallDataFit() { + Map data = new HashMap<>(); + data.put(0, new double[] {-1.0915526, 1.81983527, -0.91409478, 0.70890712, -24.55724107}); + data.put(1, new double[] {-0.61072904, 0.37545517, 0.21705352, 0.09516495, -26.57226867}); + data.put(2, new double[] {0.05485406, 0.88219898, -0.80584547, 0.94668307, 61.80919728}); + data.put(3, new double[] {-0.24835094, -0.34000053, -1.69984651, -1.45902635, -161.65525991}); + data.put(4, new double[] {0.63675392, 0.31675535, 0.38837437, -1.1221971, -14.46432611}); + data.put(5, new double[] {0.14194017, 2.18158997, -0.28397346, -0.62090588, -3.2122197}); + data.put(6, new double[] {-0.53487507, 1.4454797, 0.21570443, -0.54161422, -46.5469012}); + data.put(7, new double[] {-1.58812173, -0.73216803, -2.15670676, -1.03195988, -247.23559889}); + data.put(8, new double[] {0.20702671, 0.92864654, 0.32721202, -0.09047503, 31.61484949}); + data.put(9, new double[] {-0.37890345, -0.04846179, -0.84122753, -1.14667474, -124.92598583}); + + LinearRegressionLSQRTrainer trainer = new LinearRegressionLSQRTrainer<>(); + + LinearRegressionModel mdl = trainer.fit( + new LocalDatasetBuilder<>(data, parts), + (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1), + (k, v) -> v[4], + 4 + ); + + assertArrayEquals( + new double[]{72.26948107, 15.95144674, 24.07403921, 66.73038781}, + mdl.getWeights().getStorage().data(), + 1e-6 + ); + + assertEquals(2.8421709430404007e-14, mdl.getIntercept(), 1e-6); + } + + /** + * Tests {@code fit()} method on a big (100000 x 100) dataset. + */ + @Test + public void testBigDataFit() { + Random rnd = new Random(0); + Map data = new HashMap<>(); + double[] coef = new double[100]; + double intercept = rnd.nextDouble() * 10; + + for (int i = 0; i < 100000; i++) { + double[] x = new double[coef.length + 1]; + + for (int j = 0; j < coef.length; j++) + x[j] = rnd.nextDouble() * 10; + + x[coef.length] = intercept; + + data.put(i, x); + } + + LinearRegressionLSQRTrainer trainer = new LinearRegressionLSQRTrainer<>(); + + LinearRegressionModel mdl = trainer.fit( + new LocalDatasetBuilder<>(data, parts), + (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1), + (k, v) -> v[coef.length], + coef.length + ); + + assertArrayEquals(coef, mdl.getWeights().getStorage().data(), 1e-6); + + assertEquals(intercept, mdl.getIntercept(), 1e-6); + } +} From 5349ce7cdf3f98c4174d042920a5b0366febbd6c Mon Sep 17 00:00:00 2001 From: Anton Dmitriev Date: Thu, 8 Feb 2018 19:51:38 +0300 Subject: [PATCH 33/38] IGNITE-7438 Add LinearRegressionLSQRTrainer example. --- ...inearRegressionWithLSQRTrainerExample.java | 167 ++++++++++++++ ...LinearRegressionWithQRTrainerExample.java} | 2 +- ...inearRegressionWithSGDTrainerExample.java} | 2 +- .../ignite/ml/dataset/api/DatasetWrapper.java | 63 ----- .../ignite/ml/dataset/api/SimpleDataset.java | 216 ------------------ .../ml/dataset/api/SimpleLabeledDataset.java | 39 ---- .../builder/context/EmptyContextBuilder.java | 39 ---- .../api/builder/context/package-info.java | 22 -- .../data/SimpleDatasetDataBuilder.java | 76 ------ .../data/SimpleLabeledDatasetDataBuilder.java | 86 ------- .../api/builder/data/package-info.java | 22 -- .../ml/dataset/api/builder/package-info.java | 22 -- .../ml/dataset/api/context/EmptyContext.java | 28 --- .../ml/dataset/api/context/package-info.java | 22 -- .../dataset/api/data/SimpleDatasetData.java | 69 ------ .../api/data/SimpleLabeledDatasetData.java | 79 ------- .../ml/dataset/api/data/package-info.java | 22 -- .../ignite/ml/dataset/api/package-info.java | 22 -- .../ml/math/isolve/lsqr/AbstractLSQR.java | 70 +++--- .../ml/math/isolve/lsqr/LSQROnHeap.java | 18 +- .../isolve/lsqr/LSQRPartitionContext.java | 2 +- .../ml/math/isolve/lsqr/LSQRResult.java | 18 +- 22 files changed, 236 insertions(+), 870 deletions(-) create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java rename examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/{DistributedLinearRegressionExampleWithQRTrainer.java => DistributedLinearRegressionWithQRTrainerExample.java} (99%) rename examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/{DistributedLinearRegressionExampleWithSGDTrainer.java => DistributedLinearRegressionWithSGDTrainerExample.java} (99%) delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/DatasetWrapper.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleLabeledDataset.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/EmptyContextBuilder.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleDatasetDataBuilder.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleLabeledDatasetDataBuilder.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/EmptyContext.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleDatasetData.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleLabeledDatasetData.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/package-info.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java new file mode 100644 index 0000000000000..4cabc94c415cb --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java @@ -0,0 +1,167 @@ +/* + * 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.examples.ml.regression.linear; + +import java.util.Arrays; +import java.util.UUID; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.examples.ml.math.matrix.SparseDistributedMatrixExample; +import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; +import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; +import org.apache.ignite.ml.regressions.linear.LinearRegressionLSQRTrainer; +import org.apache.ignite.ml.regressions.linear.LinearRegressionModel; +import org.apache.ignite.thread.IgniteThread; + +/** + * Run linear regression model over distributed matrix. + * + * @see LinearRegressionLSQRTrainer + */ +public class DistributedLinearRegressionWithLSQRTrainerExample { + /** */ + private static final double[][] data = { + {8, 78, 284, 9.100000381, 109}, + {9.300000191, 68, 433, 8.699999809, 144}, + {7.5, 70, 739, 7.199999809, 113}, + {8.899999619, 96, 1792, 8.899999619, 97}, + {10.19999981, 74, 477, 8.300000191, 206}, + {8.300000191, 111, 362, 10.89999962, 124}, + {8.800000191, 77, 671, 10, 152}, + {8.800000191, 168, 636, 9.100000381, 162}, + {10.69999981, 82, 329, 8.699999809, 150}, + {11.69999981, 89, 634, 7.599999905, 134}, + {8.5, 149, 631, 10.80000019, 292}, + {8.300000191, 60, 257, 9.5, 108}, + {8.199999809, 96, 284, 8.800000191, 111}, + {7.900000095, 83, 603, 9.5, 182}, + {10.30000019, 130, 686, 8.699999809, 129}, + {7.400000095, 145, 345, 11.19999981, 158}, + {9.600000381, 112, 1357, 9.699999809, 186}, + {9.300000191, 131, 544, 9.600000381, 177}, + {10.60000038, 80, 205, 9.100000381, 127}, + {9.699999809, 130, 1264, 9.199999809, 179}, + {11.60000038, 140, 688, 8.300000191, 80}, + {8.100000381, 154, 354, 8.399999619, 103}, + {9.800000191, 118, 1632, 9.399999619, 101}, + {7.400000095, 94, 348, 9.800000191, 117}, + {9.399999619, 119, 370, 10.39999962, 88}, + {11.19999981, 153, 648, 9.899999619, 78}, + {9.100000381, 116, 366, 9.199999809, 102}, + {10.5, 97, 540, 10.30000019, 95}, + {11.89999962, 176, 680, 8.899999619, 80}, + {8.399999619, 75, 345, 9.600000381, 92}, + {5, 134, 525, 10.30000019, 126}, + {9.800000191, 161, 870, 10.39999962, 108}, + {9.800000191, 111, 669, 9.699999809, 77}, + {10.80000019, 114, 452, 9.600000381, 60}, + {10.10000038, 142, 430, 10.69999981, 71}, + {10.89999962, 238, 822, 10.30000019, 86}, + {9.199999809, 78, 190, 10.69999981, 93}, + {8.300000191, 196, 867, 9.600000381, 106}, + {7.300000191, 125, 969, 10.5, 162}, + {9.399999619, 82, 499, 7.699999809, 95}, + {9.399999619, 125, 925, 10.19999981, 91}, + {9.800000191, 129, 353, 9.899999619, 52}, + {3.599999905, 84, 288, 8.399999619, 110}, + {8.399999619, 183, 718, 10.39999962, 69}, + {10.80000019, 119, 540, 9.199999809, 57}, + {10.10000038, 180, 668, 13, 106}, + {9, 82, 347, 8.800000191, 40}, + {10, 71, 345, 9.199999809, 50}, + {11.30000019, 118, 463, 7.800000191, 35}, + {11.30000019, 121, 728, 8.199999809, 86}, + {12.80000019, 68, 383, 7.400000095, 57}, + {10, 112, 316, 10.39999962, 57}, + {6.699999809, 109, 388, 8.899999619, 94} + }; + + /** Run example. */ + public static void main(String[] args) throws InterruptedException { + System.out.println(); + System.out.println(">>> Linear regression model over sparse distributed matrix API usage example started."); + // Start ignite grid. + try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { + System.out.println(">>> Ignite grid started."); + // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread + // because we create ignite cache internally. + IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), + SparseDistributedMatrixExample.class.getSimpleName(), () -> { + IgniteCache dataCache = getTestCache(ignite); + + System.out.println(">>> Create new linear regression trainer object."); + LinearRegressionLSQRTrainer trainer = new LinearRegressionLSQRTrainer<>(); + + System.out.println(">>> Perform the training to get the model."); + LinearRegressionModel mdl = trainer.fit( + new CacheBasedDatasetBuilder<>(ignite, dataCache), + (k, v) -> Arrays.copyOfRange(v, 1, v.length), + (k, v) -> v[0], + 4 + ); + + System.out.println(">>> Linear regression model: " + mdl); + + System.out.println(">>> ---------------------------------"); + System.out.println(">>> | Prediction\t| Ground Truth\t|"); + System.out.println(">>> ---------------------------------"); + + try (QueryCursor> observations = dataCache.query(new ScanQuery<>())) { + for (Cache.Entry observation : observations) { + double[] val = observation.getValue(); + double[] inputs = Arrays.copyOfRange(val, 1, val.length); + double groundTruth = val[0]; + + double prediction = mdl.apply(new DenseLocalOnHeapVector(inputs)); + + System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth); + } + } + + System.out.println(">>> ---------------------------------"); + }); + + igniteThread.start(); + + igniteThread.join(); + } + } + + /** + * Fills cache with data and returns it. + * + * @param ignite Ignite instance. + * @return Filled Ignite Cache. + */ + private static IgniteCache getTestCache(Ignite ignite) { + CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); + cacheConfiguration.setName("TEST_" + UUID.randomUUID()); + cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 10)); + + IgniteCache cache = ignite.createCache(cacheConfiguration); + for (int i = 0; i < data.length; i++) + cache.put(i, data[i]); + return cache; + } +} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionExampleWithQRTrainer.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java similarity index 99% rename from examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionExampleWithQRTrainer.java rename to examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java index 98ff2a21166c7..2b45aa26b87bb 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionExampleWithQRTrainer.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java @@ -35,7 +35,7 @@ * * @see LinearRegressionQRTrainer */ -public class DistributedLinearRegressionExampleWithQRTrainer { +public class DistributedLinearRegressionWithQRTrainerExample { /** */ private static final double[][] data = { {8, 78, 284, 9.100000381, 109}, diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionExampleWithSGDTrainer.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java similarity index 99% rename from examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionExampleWithSGDTrainer.java rename to examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java index 3f6176251c382..f3b2655167dc4 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionExampleWithSGDTrainer.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java @@ -36,7 +36,7 @@ * * @see LinearRegressionQRTrainer */ -public class DistributedLinearRegressionExampleWithSGDTrainer { +public class DistributedLinearRegressionWithSGDTrainerExample { /** */ private static final double[][] data = { {8, 78, 284, 9.100000381, 109}, diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/DatasetWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/DatasetWrapper.java deleted file mode 100644 index 436e45761ebb1..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/DatasetWrapper.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.ml.dataset.api; - -import java.io.Serializable; -import org.apache.ignite.ml.dataset.Dataset; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; -import org.apache.ignite.ml.math.functions.IgniteTriFunction; - -/** - * A dataset wrapper that allows to introduce new functionality based on common {@code compute} methods. - * - * @param Type of a partition {@code context}. - * @param Type of a partition {@code data}. - * - * @see SimpleDataset - * @see SimpleLabeledDataset - */ -public class DatasetWrapper implements Dataset { - /** Delegate that performs {@code compute} actions. */ - protected final Dataset delegate; - - /** - * Constructs a new instance of dataset wrapper that delegates {@code compute} actions to the actual delegate. - * - * @param delegate Delegate that performs {@code compute} actions. - */ - public DatasetWrapper(Dataset delegate) { - this.delegate = delegate; - } - - /** {@inheritDoc} */ - @Override public R computeWithCtx(IgniteTriFunction map, IgniteBinaryOperator reduce, - R identity) { - return delegate.computeWithCtx(map, reduce, identity); - } - - /** {@inheritDoc} */ - @Override public R compute(IgniteBiFunction map, IgniteBinaryOperator reduce, R identity) { - return delegate.compute(map, reduce, identity); - } - - /** {@inheritDoc} */ - @Override public void close() throws Exception { - delegate.close(); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java deleted file mode 100644 index c5bf06413e26c..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleDataset.java +++ /dev/null @@ -1,216 +0,0 @@ -/* - * 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.ml.dataset.api; - -import com.github.fommil.netlib.BLAS; -import java.io.Serializable; -import org.apache.ignite.ml.dataset.Dataset; -import org.apache.ignite.ml.dataset.api.data.SimpleDatasetData; - -/** - * A simple dataset introduces additional methods based on a matrix of features. - * - * @param Type of a partition {@code context}. - */ -public class SimpleDataset extends DatasetWrapper { - /** BLAS (Basic Linear Algebra Subprograms) instance. */ - private static final BLAS blas = BLAS.getInstance(); - - /** - * Creates a new instance of simple dataset that introduces additional methods based on a matrix of features. - * - * @param delegate Delegate that performs {@code compute} actions. - */ - public SimpleDataset(Dataset delegate) { - super(delegate); - } - - /** - * Calculates mean value by all columns. - * - * @return Mean values. - */ - public double[] mean() { - ValueWithCount res = delegate.compute((data, partIdx) -> { - double[] features = data.getFeatures(); - int rows = data.getRows(); - int cols = data.getCols(); - - double[] y = new double[cols]; - - for (int col = 0; col < cols; col++) - for (int j = col * rows; j < (col + 1) * rows; j++) - y[col] += features[j]; - - return new ValueWithCount<>(y, rows); - }, (a, b) -> a == null ? b : b == null ? a : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); - - if (res != null) { - blas.dscal(res.val.length, 1.0 / res.cnt, res.val, 1); - return res.val; - } - - return null; - } - - /** - * Calculates standard deviation by all columns. - * - * @return Standard deviations. - */ - public double[] std() { - double[] mean = mean(); - ValueWithCount res = delegate.compute(data -> { - double[] features = data.getFeatures(); - int rows = data.getRows(); - int cols = data.getCols(); - - double[] y = new double[cols]; - - for (int col = 0; col < cols; col++) - for (int j = col * rows; j < (col + 1) * rows; j++) - y[col] += Math.pow(features[j] - mean[col], 2); - - return new ValueWithCount<>(y, rows); - }, (a, b) -> a == null ? b : b == null ? a : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); - - if (res != null) { - blas.dscal(res.val.length, 1.0 / res.cnt, res.val, 1); - for (int i = 0; i < res.val.length; i++) - res.val[i] = Math.sqrt(res.val[i]); - return res.val; - } - - return null; - } - - /** - * Calculates covariance matrix by all columns. - * - * @return Covariance matrix. - */ - public double[][] cov() { - double[] mean = mean(); - ValueWithCount res = delegate.compute(data -> { - double[] features = data.getFeatures(); - int rows = data.getRows(); - int cols = data.getCols(); - - double[][] y = new double[cols][cols]; - - for (int firstCol = 0; firstCol < cols; firstCol++) - for (int secondCol = 0; secondCol < cols; secondCol++) { - - for (int k = 0; k < rows; k++) { - double firstVal = features[rows * firstCol + k]; - double secondVal = features[rows * secondCol + k]; - y[firstCol][secondCol] += ((firstVal - mean[firstCol]) * (secondVal - mean[secondCol])); - } - } - - return new ValueWithCount<>(y, rows); - }, (a, b) -> a == null ? b : b == null ? a : new ValueWithCount<>(sum(a.val, b.val), a.cnt + b.cnt)); - - return res != null ? scale(res.val, 1.0 / res.cnt) : null; - } - - /** - * Calculates correlation matrix by all columns. - * - * @return Correlation matrix. - */ - public double[][] corr() { - double[][] cov = cov(); - double[] std = std(); - - for (int i = 0; i < cov.length; i++) - for (int j = 0; j < cov[0].length; j++) - cov[i][j] /= (std[i]*std[j]); - - return cov; - } - - /** - * Returns the sum of the two specified vectors. Be aware that it is in-place operation. - * - * @param a First vector. - * @param b Second vector. - * @return Sum of the two specified vectors. - */ - private static double[] sum(double[] a, double[] b) { - for (int i = 0; i < a.length; i++) - a[i] += b[i]; - - return a; - } - - /** - * Returns the sum of the two specified matrices. Be aware that it is in-place operation. - * - * @param a First matrix. - * @param b Second matrix. - * @return Sum of the two specified matrices. - */ - private static double[][] sum(double[][] a, double[][] b) { - for (int i = 0; i < a.length; i++) - for (int j = 0; j < a[i].length; j++) - a[i][j] += b[i][j]; - - return a; - } - - /** - * Multiplies all elements of the specified matrix on specified multiplier {@code alpha}. Be aware that it is - * in-place operation. - * - * @param a Matrix to be scaled. - * @param alpha Multiplier. - * @return Scaled matrix. - */ - private static double[][] scale(double[][] a, double alpha) { - for (int i = 0; i < a.length; i++) - for (int j = 0; j < a[i].length; j++) - a[i][j] *= alpha; - - return a; - } - - /** - * Util class that keeps values and count of rows this value has been calculated on. - * - * @param Type of a value. - */ - private static class ValueWithCount { - /** Value. */ - private final V val; - - /** Count of rows the value has been calculated on. */ - private final int cnt; - - /** - * Constructs a new instance of value with count. - * - * @param val Value. - * @param cnt Count of rows the value has been calculated on. - */ - ValueWithCount(V val, int cnt) { - this.val = val; - this.cnt = cnt; - } - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleLabeledDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleLabeledDataset.java deleted file mode 100644 index 7e6a876ab7196..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/SimpleLabeledDataset.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.ml.dataset.api; - -import java.io.Serializable; -import org.apache.ignite.ml.dataset.Dataset; -import org.apache.ignite.ml.dataset.api.data.SimpleLabeledDatasetData; - -/** - * A simple labeled dataset introduces additional methods based on a matrix of features and labels vector. - * - * @param Type of a partition {@code context}. - */ -public class SimpleLabeledDataset extends DatasetWrapper { - /** - * Creates a new instance of simple labeled dataset that introduces additional methods based on a matrix of features - * and labels vector. - * - * @param delegate Delegate that performs {@code compute} actions. - */ - public SimpleLabeledDataset(Dataset delegate) { - super(delegate); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/EmptyContextBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/EmptyContextBuilder.java deleted file mode 100644 index 466f7ca5a68b4..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/EmptyContextBuilder.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.ml.dataset.api.builder.context; - -import java.util.Iterator; -import org.apache.ignite.ml.dataset.PartitionContextBuilder; -import org.apache.ignite.ml.dataset.UpstreamEntry; -import org.apache.ignite.ml.dataset.api.context.EmptyContext; - -/** - * A partition {@code context} builder that makes {@link EmptyContext}. - * - * @param Type of a key in {@code upstream} data. - * @param Type of a value in {@code upstream} data. - */ -public class EmptyContextBuilder implements PartitionContextBuilder { - /** */ - private static final long serialVersionUID = 6620781747993467186L; - - /** {@inheritDoc} */ - @Override public EmptyContext build(Iterator> upstreamData, long upstreamDataSize) { - return new EmptyContext(); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/package-info.java deleted file mode 100644 index 166a02da98baa..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/context/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Contains partition {@code context} builders. - */ -package org.apache.ignite.ml.dataset.api.builder.context; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleDatasetDataBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleDatasetDataBuilder.java deleted file mode 100644 index 95ec5a8781d35..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleDatasetDataBuilder.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * 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.ml.dataset.api.builder.data; - -import java.io.Serializable; -import java.util.Iterator; -import org.apache.ignite.ml.dataset.PartitionDataBuilder; -import org.apache.ignite.ml.dataset.UpstreamEntry; -import org.apache.ignite.ml.dataset.api.data.SimpleDatasetData; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; - -/** - * A partition {@code data} builder that makes {@link SimpleDatasetData}. - * - * @param Type of a key in upstream data. - * @param Type of a value in upstream data. - * @param Type of a partition context. - */ -public class SimpleDatasetDataBuilder - implements PartitionDataBuilder { - /** */ - private static final long serialVersionUID = 756800193212149975L; - - /** Function that extracts features from an {@code upstream} data. */ - private final IgniteBiFunction featureExtractor; - - /** Number of columns (features). */ - private final int cols; - - /** - * Construct a new instance of partition {@code data} builder that makes {@link SimpleDatasetData}. - * - * @param featureExtractor Function that extracts features from an {@code upstream} data. - * @param cols Number of columns (features). - */ - public SimpleDatasetDataBuilder(IgniteBiFunction featureExtractor, int cols) { - this.featureExtractor = featureExtractor; - this.cols = cols; - } - - /** {@inheritDoc} */ - @Override public SimpleDatasetData build(Iterator> upstreamData, long upstreamDataSize, C ctx) { - // Prepares the matrix of features in flat column-major format. - double[] features = new double[Math.toIntExact(upstreamDataSize * cols)]; - - int ptr = 0; - while (upstreamData.hasNext()) { - UpstreamEntry entry = upstreamData.next(); - double[] row = featureExtractor.apply(entry.getKey(), entry.getValue()); - - assert row.length == cols : "Feature extractor must return exactly " + cols + " features"; - - for (int i = 0; i < cols; i++) - features[Math.toIntExact(i * upstreamDataSize + ptr)] = row[i]; - - ptr++; - } - - return new SimpleDatasetData(features, Math.toIntExact(upstreamDataSize), cols); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleLabeledDatasetDataBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleLabeledDatasetDataBuilder.java deleted file mode 100644 index 0765daa027b75..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/SimpleLabeledDatasetDataBuilder.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * 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.ml.dataset.api.builder.data; - -import java.io.Serializable; -import java.util.Iterator; -import org.apache.ignite.ml.dataset.PartitionDataBuilder; -import org.apache.ignite.ml.dataset.UpstreamEntry; -import org.apache.ignite.ml.dataset.api.data.SimpleLabeledDatasetData; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; - -/** - * A partition {@code data} builder that makes {@link SimpleLabeledDatasetData}. - * - * @param Type of a key in upstream data. - * @param Type of a value in upstream data. - * @param type of a partition context. - */ -public class SimpleLabeledDatasetDataBuilder - implements PartitionDataBuilder { - /** */ - private static final long serialVersionUID = 3678784980215216039L; - - /** Function that extracts features from an {@code upstream} data. */ - private final IgniteBiFunction featureExtractor; - - /** Function that extracts labels from an {@code upstream} data. */ - private final IgniteBiFunction lbExtractor; - - /** Number of columns (features). */ - private final int cols; - - /** - * Constructs a new instance of partition {@code data} builder that makes {@link SimpleLabeledDatasetData}. - * - * @param featureExtractor Function that extracts features from an {@code upstream} data. - * @param lbExtractor Function that extracts labels from an {@code upstream} data. - * @param cols Number of columns (features). - */ - public SimpleLabeledDatasetDataBuilder(IgniteBiFunction featureExtractor, - IgniteBiFunction lbExtractor, int cols) { - this.featureExtractor = featureExtractor; - this.lbExtractor = lbExtractor; - this.cols = cols; - } - - /** {@inheritDoc} */ - @Override public SimpleLabeledDatasetData build(Iterator> upstreamData, - long upstreamDataSize, C ctx) { - // Prepares the matrix of features in flat column-major format. - double[] features = new double[Math.toIntExact(upstreamDataSize * cols)]; - double[] labels = new double[Math.toIntExact(upstreamDataSize)]; - - int ptr = 0; - while (upstreamData.hasNext()) { - UpstreamEntry entry = upstreamData.next(); - double[] row = featureExtractor.apply(entry.getKey(), entry.getValue()); - - assert row.length == cols : "Feature extractor must return exactly " + cols + " features"; - - for (int i = 0; i < cols; i++) - features[Math.toIntExact(i * upstreamDataSize) + ptr] = row[i]; - - labels[ptr] = lbExtractor.apply(entry.getKey(), entry.getValue()); - - ptr++; - } - - return new SimpleLabeledDatasetData(features, Math.toIntExact(upstreamDataSize), cols, labels); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/package-info.java deleted file mode 100644 index a019c762873be..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/data/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Contains partition {@code data} builders. - */ -package org.apache.ignite.ml.dataset.api.builder.data; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/package-info.java deleted file mode 100644 index 55310426b8d27..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/builder/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Base package for partition {@code data} and {@code context} builders. - */ -package org.apache.ignite.ml.dataset.api.builder; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/EmptyContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/EmptyContext.java deleted file mode 100644 index 795be142d7e60..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/EmptyContext.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * 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.ml.dataset.api.context; - -import java.io.Serializable; - -/** - * An empty partition {@code context}. - */ -public class EmptyContext implements Serializable { - /** */ - private static final long serialVersionUID = 4108938672110578991L; -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/package-info.java deleted file mode 100644 index 4f376d9569240..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/context/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Contains implementation of partition {@code context}. - */ -package org.apache.ignite.ml.dataset.api.context; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleDatasetData.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleDatasetData.java deleted file mode 100644 index 60d599cbbf730..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleDatasetData.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.ml.dataset.api.data; - -import org.apache.ignite.ml.dataset.api.SimpleDataset; - -/** - * A partition {@code data} of the {@link SimpleDataset} containing matrix of features in flat column-major format - * stored in heap. - */ -public class SimpleDatasetData implements AutoCloseable { - /** Matrix of features in a dense flat column-major format. */ - private final double[] features; - - /** Number of rows. */ - private final int rows; - - /** Number of columns. */ - private final int cols; - - /** - * Constructs a new instance of partition {@code data} of the {@link SimpleDataset} containing matrix of features in - * flat column-major format stored in heap. - * - * @param features Matrix of features in a dense flat column-major format. - * @param rows Number of rows. - * @param cols Number of columns. - */ - public SimpleDatasetData(double[] features, int rows, int cols) { - this.features = features; - this.rows = rows; - this.cols = cols; - } - - /** */ - public double[] getFeatures() { - return features; - } - - /** */ - public int getRows() { - return rows; - } - - /** */ - public int getCols() { - return cols; - } - - /** {@inheritDoc} */ - @Override public void close() { - // Do nothing, GC will clean up. - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleLabeledDatasetData.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleLabeledDatasetData.java deleted file mode 100644 index d326c0672c023..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/SimpleLabeledDatasetData.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.ml.dataset.api.data; - -import org.apache.ignite.ml.dataset.api.SimpleLabeledDataset; - -/** - * A partition {@code data} of the {@link SimpleLabeledDataset} containing matrix of features in flat column-major - * format stored in heap and vector of labels stored in heap as well. - */ -public class SimpleLabeledDatasetData implements AutoCloseable { - /** Matrix with features in a dense flat column-major format. */ - private final double[] features; - - /** Number of rows. */ - private final int rows; - - /** Number of columns. */ - private final int cols; - - /** Vector with labels. */ - private final double[] labels; - - /** - * Constructs a new instance of partition {@code data} of the {@link SimpleLabeledDataset} containing matrix of - * features in flat column-major format stored in heap and vector of labels stored in heap as well. - * - * @param features Matrix with features in a dense flat column-major format. - * @param rows Number of rows. - * @param cols Number of columns. - * @param labels Vector with labels. - */ - public SimpleLabeledDatasetData(double[] features, int rows, int cols, double[] labels) { - this.features = features; - this.rows = rows; - this.cols = cols; - this.labels = labels; - } - - /** */ - public double[] getFeatures() { - return features; - } - - /** */ - public int getRows() { - return rows; - } - - /** */ - public int getCols() { - return cols; - } - - /** */ - public double[] getLabels() { - return labels; - } - - /** {@inheritDoc} */ - @Override public void close() { - // Do nothing, GC will clean up. - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/package-info.java deleted file mode 100644 index 83eca8be8d7a0..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/data/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Contains implementation of partition {@code data}. - */ -package org.apache.ignite.ml.dataset.api.data; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/package-info.java deleted file mode 100644 index 19c051c00cc22..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/api/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ - -/** - * - * Base package for machine learning dataset classes representing dataset API. - */ -package org.apache.ignite.ml.dataset.api; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java index 070f9c08c356b..220702cc0e773 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java @@ -25,34 +25,28 @@ * device. */ public abstract class AbstractLSQR { - /** - * The smallest representable positive number such that 1.0 + eps != 1.0. - */ + /** The smallest representable positive number such that 1.0 + eps != 1.0. */ private static final double eps = Double.longBitsToDouble(Double.doubleToLongBits(1.0) | 1) - 1.0; - /** */ + /** BLAS (Basic Linear Algebra Subprograms) instance. */ private static BLAS blas = BLAS.getInstance(); /** * Solves given Sparse Linear Systems. * - * @param damp damping coefficient - * @param atol stopping tolerances, if both (atol and btol) are 1.0e-9 (say), the final residual norm should be - * accurate to about 9 digits. (The final x will usually have fewer correct digits, depending on cond(A) and the - * size of damp.) - * @param btol stopping tolerances, if both (atol and btol) are 1.0e-9 (say), the final residual norm should be - * accurate to about 9 digits. (The final x will usually have fewer correct digits, depending on cond(A) and the - * size of damp.) - * @param conlim another stopping tolerance, lsqr terminates if an estimate of cond(A) exceeds conlim. For - * compatible systems Ax = b, conlim could be as large as 1.0e+12 (say). For least-squares problems, conlim should - * be less than 1.0e+8. Maximum precision can be obtained by setting atol = btol = conlim = zero, but the number of - * iterations may then be excessive. - * @param iterLim explicit limitation on number of iterations (for safety) - * @param calcVar whether to estimate diagonals of (A'A + damp^2*I)^{-1} - * @param x0 initial value of x - * @return solver result + * @param damp Damping coefficient. + * @param atol Stopping tolerances, if both (atol and btol) are 1.0e-9 (say), the final residual norm should be + * accurate to about 9 digits. + * @param btol Stopping tolerances, if both (atol and btol) are 1.0e-9 (say), the final residual norm should be + * accurate to about 9 digits. + * @param conlim Another stopping tolerance, LSQR terminates if an estimate of cond(A) exceeds conlim. + * @param iterLim Explicit limitation on number of iterations (for safety). + * @param calcVar Whether to estimate diagonals of (A'A + damp^2*I)^{-1}. + * @param x0 Initial value of x. + * @return Solver result. */ - public LSQRResult solve(double damp, double atol, double btol, double conlim, double iterLim, boolean calcVar, double[] x0) { + public LSQRResult solve(double damp, double atol, double btol, double conlim, double iterLim, boolean calcVar, + double[] x0) { int n = getColumns(); if (iterLim < 0) iterLim = 2 * n; @@ -164,7 +158,7 @@ public LSQRResult solve(double damp, double atol, double btol, double conlim, do ddnorm = ddnorm + Math.pow(blas.dnrm2(dk.length, dk, 1), 2); if (calcVar) - blas.daxpy(var.length, 1.0, pow(dk, 2), 1, var, 1); + blas.daxpy(var.length, 1.0, pow2(dk), 1, var, 1); // Use a plane rotation on the right to eliminate the // super-diagonal element (theta) of the upper-bidiagonal matrix. @@ -238,13 +232,30 @@ public LSQRResult solve(double damp, double atol, double btol, double conlim, do return new LSQRResult(x, itn, istop, r1norm, r2norm, anorm, acond, arnorm, xnorm, var); } - /** */ + /** + * Calculates bnorm. + * + * @return bnorm + */ protected abstract double bnorm(); - /** */ + /** + * Calculates beta. + * + * @param x X value. + * @param alfa Alfa value. + * @param beta Beta value. + * @return Beta. + */ protected abstract double beta(double[] x, double alfa, double beta); - /** */ + /** + * Perform LSQR iteration. + * + * @param bnorm Bnorm value. + * @param target Target value. + * @return Iteration result. + */ protected abstract double[] iter(double bnorm, double[] target); /** */ @@ -253,9 +264,9 @@ public LSQRResult solve(double damp, double atol, double btol, double conlim, do /** */ private static double[] symOrtho(double a, double b) { if (b == 0) - return new double[]{ Math.signum(a), 0, Math.abs(a) }; + return new double[] {Math.signum(a), 0, Math.abs(a)}; else if (a == 0) - return new double[]{ 0, Math.signum(b), Math.abs(b) }; + return new double[] {0, Math.signum(b), Math.abs(b)}; else { double c, s, r; if (Math.abs(b) > Math.abs(a)) { @@ -270,7 +281,7 @@ else if (a == 0) s = c * tau; r = a / c; } - return new double[]{ c, s, r }; + return new double[] {c, s, r}; } } @@ -279,13 +290,12 @@ else if (a == 0) * it's "in place" operation. * * @param a Vector or matrix of doubles. - * @param pow Power. * @return Matrix with elements raised to the specified power. */ - private static double[] pow(double[] a, double pow) { + private static double[] pow2(double[] a) { double[] res = new double[a.length]; for (int i = 0; i < res.length; i++) - res[i] = Math.pow(a[i], pow); + res[i] = Math.pow(a[i], 2); return res; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java index 75688022b274a..73f674ae0d307 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java @@ -28,7 +28,7 @@ * Distributed implementation of LSQR algorithm based on. */ public class LSQROnHeap extends AbstractLSQR implements AutoCloseable { - + /** Dataset. */ private final Dataset dataset; /** @@ -47,20 +47,21 @@ public LSQROnHeap(DatasetBuilder datasetBuilder, /** {@inheritDoc} */ @Override protected double bnorm() { - double res = dataset.computeWithCtx((ctx, data) -> { + return dataset.computeWithCtx((ctx, data) -> { ctx.setU(Arrays.copyOf(data.getY(), data.getY().length)); + return BLAS.getInstance().dnrm2(data.getY().length, data.getY(), 1); }, (a, b) -> a == null ? b : Math.sqrt(a * a + b * b)); - return res; } /** {@inheritDoc} */ @Override protected double beta(double[] x, double alfa, double beta) { - double bb = dataset.computeWithCtx((ctx, data) -> { - BLAS.getInstance().dgemv("N", data.getRows(), data.getCols(), alfa, data.getX(), data.getRows(), x, 1, beta, ctx.getU(), 1); + return dataset.computeWithCtx((ctx, data) -> { + BLAS.getInstance().dgemv("N", data.getRows(), data.getCols(), alfa, data.getX(), + Math.max(1, data.getRows()), x, 1, beta, ctx.getU(), 1); + return BLAS.getInstance().dnrm2(ctx.getU().length, ctx.getU(), 1); }, (a, b) -> a == null ? b : Math.sqrt(a * a + b * b)); - return bb; } /** {@inheritDoc} */ @@ -68,13 +69,16 @@ public LSQROnHeap(DatasetBuilder datasetBuilder, double[] res = dataset.computeWithCtx((ctx, data) -> { BLAS.getInstance().dscal(ctx.getU().length, 1 / bnorm, ctx.getU(), 1); double[] v = new double[data.getCols()]; - BLAS.getInstance().dgemv("T", data.getRows(), data.getCols(), 1.0, data.getX(), data.getRows(), ctx.getU(), 1, 0, v, 1); + BLAS.getInstance().dgemv("T", data.getRows(), data.getCols(), 1.0, data.getX(), + Math.max(1, data.getRows()), ctx.getU(), 1, 0, v, 1); + return v; }, (a, b) -> { if (a == null) return b; else { BLAS.getInstance().daxpy(a.length, 1.0, a, 1, b, 1); + return b; } }); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRPartitionContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRPartitionContext.java index 5769cfb5a5279..0ec9805ee4432 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRPartitionContext.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRPartitionContext.java @@ -26,7 +26,7 @@ public class LSQRPartitionContext implements Serializable { /** */ private static final long serialVersionUID = -8159608186899430315L; - /** Part of V vector. */ + /** Part of U vector. */ private double[] u; /** */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRResult.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRResult.java index d320ea677227b..f0c552e932b7d 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRResult.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRResult.java @@ -65,12 +65,24 @@ public class LSQRResult extends IterativeSolverResult { /** * If calc_var is True, estimates all diagonals of (A'A)^{-1} (if damp == 0) or more generally - * (A'A + damp^2*I)^{-1}. This is well defined if A has full column rank or damp > 0. (Not sure what var means if - * rank(A) < n and damp = 0.) + * (A'A + damp^2*I)^{-1}. This is well defined if A has full column rank or damp > 0. */ private final double[] var; - /** */ + /** + * Constructs a new instance of LSQR result. + * + * @param x X value. + * @param iterations Number of performed iterations. + * @param isstop Stop reason. + * @param r1norn R1 norm value. + * @param r2norm R2 norm value. + * @param anorm A norm value. + * @param acond A cond value. + * @param arnorm AR norm value. + * @param xnorm X norm value. + * @param var Var value. + */ public LSQRResult(double[] x, int iterations, int isstop, double r1norn, double r2norm, double anorm, double acond, double arnorm, double xnorm, double[] var) { super(x, iterations); From 0b466eaa75288d3d38eef9fbadb6b5c5f91518e6 Mon Sep 17 00:00:00 2001 From: Anton Dmitriev Date: Thu, 8 Feb 2018 20:01:58 +0300 Subject: [PATCH 34/38] IGNITE-7438 Refactor javadoc. --- ...inearRegressionWithLSQRTrainerExample.java | 3 + .../org/apache/ignite/ml/DatasetTrainer.java | 2 +- .../ml/math/isolve/lsqr/LSQRResult.java | 24 ++--- .../ml/dataset/api/DatasetWrapperTest.java | 87 ------------------- 4 files changed, 10 insertions(+), 106 deletions(-) delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dataset/api/DatasetWrapperTest.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java index 4cabc94c415cb..20e0653623bc1 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java @@ -104,6 +104,7 @@ public static void main(String[] args) throws InterruptedException { // Start ignite grid. try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { System.out.println(">>> Ignite grid started."); + // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread // because we create ignite cache internally. IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), @@ -160,8 +161,10 @@ private static IgniteCache getTestCache(Ignite ignite) { cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 10)); IgniteCache cache = ignite.createCache(cacheConfiguration); + for (int i = 0; i < data.length; i++) cache.put(i, data[i]); + return cache; } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/DatasetTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/DatasetTrainer.java index d1a68d5b3706c..aa04d8e4f0dd3 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/DatasetTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/DatasetTrainer.java @@ -21,7 +21,7 @@ import org.apache.ignite.ml.math.functions.IgniteBiFunction; /** - * Interface for Trainers. Trainer is just a function which produces model from the data. + * Interface for trainers. Trainer is just a function which produces model from the data. * * @param Type of a key in {@code upstream} data. * @param Type of a value in {@code upstream} data. diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRResult.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRResult.java index f0c552e932b7d..47beddbef2ca0 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRResult.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQRResult.java @@ -33,34 +33,22 @@ public class LSQRResult extends IterativeSolverResult { */ private final int isstop; - /** - * Represents norm(r), where r = b - Ax. - */ + /** Represents norm(r), where r = b - Ax. */ private final double r1norn; - /** - * Represents sqrt( norm(r)^2 + damp^2 * norm(x)^2 ). Equal to r1norm if damp == 0. - */ + /**Represents sqrt( norm(r)^2 + damp^2 * norm(x)^2 ). Equal to r1norm if damp == 0. */ private final double r2norm; - /** - * Estimate of Frobenius norm of Abar = [[A]; [damp*I]]. - */ + /** Estimate of Frobenius norm of Abar = [[A]; [damp*I]]. */ private final double anorm; - /** - * Estimate of cond(Abar). - */ + /** Estimate of cond(Abar). */ private final double acond; - /** - * Estimate of norm(A'*r - damp^2*x). - */ + /** Estimate of norm(A'*r - damp^2*x). */ private final double arnorm; - /** - * Represents norm(x). - */ + /** Represents norm(x). */ private final double xnorm; /** diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/api/DatasetWrapperTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/api/DatasetWrapperTest.java deleted file mode 100644 index ca5cf4ca98865..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/api/DatasetWrapperTest.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * 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.ml.dataset.api; - -import java.io.Serializable; -import org.apache.ignite.ml.dataset.Dataset; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteBinaryOperator; -import org.apache.ignite.ml.math.functions.IgniteTriFunction; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.runners.MockitoJUnitRunner; - -import static org.junit.Assert.assertEquals; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; - -/** - * Tests for {@link DatasetWrapper}. - */ -@RunWith(MockitoJUnitRunner.class) -public class DatasetWrapperTest { - /** Mocked dataset. */ - @Mock - private Dataset dataset; - - /** Dataset wrapper. */ - private DatasetWrapper wrapper; - - /** Initialization. */ - @Before - public void beforeTest() { - wrapper = new DatasetWrapper<>(dataset); - } - - /** Tests {@code computeWithCtx()} method. */ - @Test - @SuppressWarnings("unchecked") - public void testComputeWithCtx() { - doReturn(42).when(dataset).computeWithCtx(any(IgniteTriFunction.class), any(), any()); - - Integer res = wrapper.computeWithCtx(mock(IgniteTriFunction.class), mock(IgniteBinaryOperator.class), null); - - assertEquals(42, res.intValue()); - verify(dataset, times(1)).computeWithCtx(any(IgniteTriFunction.class), any(), any()); - } - - /** Tests {@code compute()} method. */ - @Test - @SuppressWarnings("unchecked") - public void testCompute() { - doReturn(42).when(dataset).compute(any(IgniteBiFunction.class), any(), any()); - - Integer res = wrapper.compute(mock(IgniteBiFunction.class), mock(IgniteBinaryOperator.class), null); - - assertEquals(42, res.intValue()); - verify(dataset, times(1)).compute(any(IgniteBiFunction.class), any(), any()); - } - - /** Tests {@code close()} method. */ - @Test - public void testClose() throws Exception { - wrapper.close(); - - verify(dataset, times(1)).close(); - } -} From 802af6a9210a9c9cf66cb237c35eb37eb3f84eca Mon Sep 17 00:00:00 2001 From: Anton Dmitriev Date: Thu, 8 Feb 2018 20:06:06 +0300 Subject: [PATCH 35/38] IGNITE-7438 Add more tests. --- .../ml/math/isolve/lsqr/LSQROnHeapTest.java | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java index 5c080886f87e5..4892ff80725a5 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java @@ -76,6 +76,31 @@ public void testSolveLinearSystem() { assertArrayEquals(new double[]{1, -2, -2}, res.getX(), 1e-6); } + /** Tests solving simple linear system with specified x0. */ + @Test + public void testSolveLinearSystemWithX0() { + Map data = new HashMap<>(); + data.put(0, new double[]{3, 2, -1, 1}); + data.put(1, new double[]{2, -2, 4, -2}); + data.put(2, new double[]{-1, 0.5, -1, 0}); + + DatasetBuilder datasetBuilder = new LocalDatasetBuilder<>(data, parts); + + LSQROnHeap lsqr = new LSQROnHeap<>( + datasetBuilder, + new LinSysPartitionDataBuilderOnHeap<>( + (k, v) -> Arrays.copyOf(v, v.length - 1), + (k, v) -> v[3], + 3 + ) + ); + + LSQRResult res = lsqr.solve(0, 1e-12, 1e-12, 1e8, -1, false, + new double[] {999, 999, 999}); + + assertArrayEquals(new double[]{1, -2, -2}, res.getX(), 1e-6); + } + /** Tests solving least squares problem. */ @Test public void testSolveLeastSquares() throws Exception { From a649e878842505183f3f6d2e095fcf617446a009 Mon Sep 17 00:00:00 2001 From: Anton Dmitriev Date: Fri, 9 Feb 2018 14:04:12 +0300 Subject: [PATCH 36/38] IGNITE-7438 Update javadoc and reformat code. --- .../java/org/apache/ignite/ml/Trainer.java | 1 + .../ml/math/isolve/lsqr/AbstractLSQR.java | 30 +++++++++++++++++++ .../ml/math/isolve/lsqr/LSQROnHeap.java | 2 +- .../apache/ignite/ml/trainers/Trainer.java | 1 + 4 files changed, 33 insertions(+), 1 deletion(-) diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java index a95a1cc10e966..bbf1e1e3d36e3 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java @@ -26,6 +26,7 @@ * @param Type of produced model. * @param Type of data needed for model producing. */ +@Deprecated public interface Trainer { /** * Returns model based on data diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java index 220702cc0e773..0e08fab435e10 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java @@ -23,6 +23,9 @@ /** * Basic implementation of the LSQR algorithm without assumptions about dataset storage format or data processing * device. + * + * This implementation is based of SciPy implementation. + * SciPy implementation: https://github.com/scipy/scipy/blob/master/scipy/sparse/linalg/isolve/lsqr.py#L98. */ public abstract class AbstractLSQR { /** The smallest representable positive number such that 1.0 + eps != 1.0. */ @@ -48,14 +51,18 @@ public abstract class AbstractLSQR { public LSQRResult solve(double damp, double atol, double btol, double conlim, double iterLim, boolean calcVar, double[] x0) { int n = getColumns(); + if (iterLim < 0) iterLim = 2 * n; + double[] var = new double[n]; int itn = 0; int istop = 0; double ctol = 0; + if (conlim > 0) ctol = 1 / conlim; + double anorm = 0; double acond = 0; double dampsq = Math.pow(damp, 2.0); @@ -72,6 +79,7 @@ public LSQRResult solve(double damp, double atol, double btol, double conlim, do double bnorm = bnorm(); double[] x; double beta; + if (x0 == null) { x = new double[n]; beta = bnorm; @@ -80,8 +88,10 @@ public LSQRResult solve(double damp, double atol, double btol, double conlim, do x = x0; beta = beta(x, -1.0, 1.0); } + double[] v = new double[n]; double alfa; + if (beta > 0) { v = iter(beta, v); alfa = blas.dnrm2(v.length, v, 1); @@ -93,6 +103,7 @@ public LSQRResult solve(double damp, double atol, double btol, double conlim, do if (alfa > 0) blas.dscal(v.length, 1 / alfa, v, 1); + double[] w = Arrays.copyOf(v, v.length); double rhobar = alfa; @@ -102,6 +113,7 @@ public LSQRResult solve(double damp, double atol, double btol, double conlim, do double r2norm = rnorm; double arnorm = alfa * beta; double[] dk = new double[w.length]; + if (arnorm == 0) return new LSQRResult(x, itn, istop, r1norm, r2norm, anorm, acond, arnorm, xnorm, var); @@ -116,10 +128,14 @@ public LSQRResult solve(double damp, double atol, double btol, double conlim, do beta = beta(v, 1.0, -alfa); if (beta > 0) { anorm = Math.sqrt(Math.pow(anorm, 2) + Math.pow(alfa, 2) + Math.pow(beta, 2) + Math.pow(damp, 2)); + blas.dscal(v.length, -beta, v, 1); + iter(beta, v); + //v = dataset.iter(beta, n); alfa = blas.dnrm2(v.length, v, 1); + if (alfa > 0) blas.dscal(v.length, 1 / alfa, v, 1); } @@ -147,6 +163,7 @@ public LSQRResult solve(double damp, double atol, double btol, double conlim, do double t1 = phi / rho; double t2 = -theta / rho; + blas.dcopy(w.length, w, 1, dk, 1); blas.dscal(dk.length, 1 / rho, dk, 1); @@ -155,6 +172,7 @@ public LSQRResult solve(double damp, double atol, double btol, double conlim, do // w = v + t2*w blas.dscal(w.length, t2, w, 1); blas.daxpy(w.length, 1, v, 1, w, 1); + ddnorm = ddnorm + Math.pow(blas.dnrm2(dk.length, dk, 1), 2); if (calcVar) @@ -192,8 +210,10 @@ public LSQRResult solve(double damp, double atol, double btol, double conlim, do // Although there is cancellation, it might be accurate enough. double r1sq = Math.pow(rnorm, 2) - dampsq * xxnorm; r1norm = Math.sqrt(Math.abs(r1sq)); + if (r1sq < 0) r1norm = -r1norm; + r2norm = rnorm; // Now use these norms to estimate certain other quantities, @@ -211,24 +231,30 @@ public LSQRResult solve(double damp, double atol, double btol, double conlim, do // atol = eps, btol = eps, conlim = 1/eps. if (itn >= iterLim) istop = 7; + if (1 + test3 <= 1) istop = 6; + if (1 + test2 <= 1) istop = 5; + if (1 + t1 <= 1) istop = 4; // Allow for tolerances set by the user. if (test3 <= ctol) istop = 3; + if (test2 <= atol) istop = 2; + if (test1 <= rtol) istop = 1; if (istop != 0) break; } + return new LSQRResult(x, itn, istop, r1norm, r2norm, anorm, acond, arnorm, xnorm, var); } @@ -269,6 +295,7 @@ else if (a == 0) return new double[] {0, Math.signum(b), Math.abs(b)}; else { double c, s, r; + if (Math.abs(b) > Math.abs(a)) { double tau = a / b; s = Math.signum(b) / Math.sqrt(1 + tau * tau); @@ -281,6 +308,7 @@ else if (a == 0) s = c * tau; r = a / c; } + return new double[] {c, s, r}; } } @@ -294,8 +322,10 @@ else if (a == 0) */ private static double[] pow2(double[] a) { double[] res = new double[a.length]; + for (int i = 0; i < res.length; i++) res[i] = Math.pow(a[i], 2); + return res; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java index 73f674ae0d307..fa8e7132ecd96 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java @@ -25,7 +25,7 @@ import org.apache.ignite.ml.math.isolve.LinSysPartitionDataOnHeap; /** - * Distributed implementation of LSQR algorithm based on. + * Distributed implementation of LSQR algorithm based on {@link AbstractLSQR} and {@link Dataset}. */ public class LSQROnHeap extends AbstractLSQR implements AutoCloseable { /** Dataset. */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java index 5efdf57dcbad9..71350baad058e 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java @@ -20,6 +20,7 @@ import org.apache.ignite.ml.Model; /** Trainer interface. */ +@Deprecated public interface Trainer { /** * Train the model based on provided data. From bb9d905f7a764c9d1358f38ba3d2f4c4fa882cf9 Mon Sep 17 00:00:00 2001 From: Anton Dmitriev Date: Fri, 9 Feb 2018 14:12:32 +0300 Subject: [PATCH 37/38] IGNITE-7438 Update javadoc and reformat code, added "TODO". --- modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java | 1 + .../org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java | 3 ++- .../src/main/java/org/apache/ignite/ml/trainers/Trainer.java | 1 + 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java index bbf1e1e3d36e3..4e0a5704473ef 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java @@ -26,6 +26,7 @@ * @param Type of produced model. * @param Type of data needed for model producing. */ +// TODO: IGNITE-7659: Reduce multiple Trainer interfaces to one @Deprecated public interface Trainer { /** diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java index 0e08fab435e10..8d190cd74373a 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java @@ -24,9 +24,10 @@ * Basic implementation of the LSQR algorithm without assumptions about dataset storage format or data processing * device. * - * This implementation is based of SciPy implementation. + * This implementation is based on SciPy implementation. * SciPy implementation: https://github.com/scipy/scipy/blob/master/scipy/sparse/linalg/isolve/lsqr.py#L98. */ +// TODO: IGNITE-7660: Refactor LSQR algorithm public abstract class AbstractLSQR { /** The smallest representable positive number such that 1.0 + eps != 1.0. */ private static final double eps = Double.longBitsToDouble(Double.doubleToLongBits(1.0) | 1) - 1.0; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java index 71350baad058e..b4f83d9e23acb 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java @@ -21,6 +21,7 @@ /** Trainer interface. */ @Deprecated +// TODO: IGNITE-7659: Reduce multiple Trainer interfaces to one public interface Trainer { /** * Train the model based on provided data. From 98ccf03056b43a2250246efb108ccf9c25ec3166 Mon Sep 17 00:00:00 2001 From: Anton Dmitriev Date: Fri, 9 Feb 2018 14:17:10 +0300 Subject: [PATCH 38/38] IGNITE-7438 Update javadoc and reformat code. --- modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java | 2 +- .../org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java | 2 +- .../ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java index 4e0a5704473ef..f29afa2d797f3 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java @@ -26,7 +26,7 @@ * @param Type of produced model. * @param Type of data needed for model producing. */ -// TODO: IGNITE-7659: Reduce multiple Trainer interfaces to one +// TODO: IGNITE-7659: Reduce multiple Trainer interfaces to one. @Deprecated public interface Trainer { /** diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java index 8d190cd74373a..f704c805d3941 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java @@ -27,7 +27,7 @@ * This implementation is based on SciPy implementation. * SciPy implementation: https://github.com/scipy/scipy/blob/master/scipy/sparse/linalg/isolve/lsqr.py#L98. */ -// TODO: IGNITE-7660: Refactor LSQR algorithm +// TODO: IGNITE-7660: Refactor LSQR algorithm. public abstract class AbstractLSQR { /** The smallest representable positive number such that 1.0 + eps != 1.0. */ private static final double eps = Double.longBitsToDouble(Double.doubleToLongBits(1.0) | 1) - 1.0; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java index b4f83d9e23acb..8fd96931b23e9 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java @@ -21,7 +21,7 @@ /** Trainer interface. */ @Deprecated -// TODO: IGNITE-7659: Reduce multiple Trainer interfaces to one +// TODO: IGNITE-7659: Reduce multiple Trainer interfaces to one. public interface Trainer { /** * Train the model based on provided data.