Repository: incubator-ignite Updated Branches: refs/heads/ignite-1198 023ffe0ee -> 1402164df
IGNITE-1198 - Bring IGNITE_HOME from driver node, use correct number of executors. Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/1402164d Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/1402164d Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/1402164d Branch: refs/heads/ignite-1198 Commit: 1402164df76ba8b2d6528618860fcb0a17df8e0f Parents: 023ffe0 Author: Alexey Goncharuk <agoncha...@gridgain.com> Authored: Wed Aug 5 12:47:12 2015 -0700 Committer: Alexey Goncharuk <agoncha...@gridgain.com> Committed: Wed Aug 5 12:47:12 2015 -0700 ---------------------------------------------------------------------- .../org/apache/ignite/spark/IgniteContext.scala | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1402164d/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala ---------------------------------------------------------------------- diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala index 5267244a..115e34b 100644 --- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala +++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala @@ -19,7 +19,8 @@ package org.apache.ignite.spark import org.apache.ignite.internal.IgnitionEx -import org.apache.ignite.{Ignition, Ignite} +import org.apache.ignite.internal.util.IgniteUtils +import org.apache.ignite.{IgniteSystemProperties, Ignition, Ignite} import org.apache.ignite.configuration.{CacheConfiguration, IgniteConfiguration} import org.apache.spark.{Logging, SparkContext} import org.apache.spark.sql.SQLContext @@ -41,8 +42,11 @@ class IgniteContext[K, V]( private val cfgClo = new Once(cfgF) + private val igniteHome = IgniteUtils.getIgniteHome + if (!client) { - val workers = sparkContext.getExecutorStorageStatus.length - 1 + // Minimum 1 executor is required for any Spark job. + val workers = sparkContext.getConf.getInt("spark.executor.instances", 1) if (workers <= 0) throw new IllegalStateException("No Spark executors found to start Ignite nodes.") @@ -125,6 +129,16 @@ class IgniteContext[K, V]( * @return Ignite instance. */ def ignite(): Ignite = { + val home = IgniteUtils.getIgniteHome + + if (home == null && igniteHome != null) { + logInfo("Setting IGNITE_HOME from driver not as it is not available on this worker: " + igniteHome) + + IgniteUtils.nullifyHomeDirectory() + + System.setProperty(IgniteSystemProperties.IGNITE_HOME, igniteHome) + } + val igniteCfg = cfgClo() try {