From 7a1ace94292bd788bd2a0a4fd470c9584629a1d1 Mon Sep 17 00:00:00 2001 From: Hemant Bhanawat Date: Sat, 16 Apr 2016 23:43:32 -0700 Subject: [PATCH 001/142] [SPARK-13904][SCHEDULER] Add support for pluggable cluster manager This commit adds support for pluggable cluster manager. And also allows a cluster manager to clean up tasks without taking the parent process down. To plug a new external cluster manager, ExternalClusterManager trait should be implemented. It returns task scheduler and backend scheduler that will be used by SparkContext to schedule tasks. An external cluster manager is registered using the java.util.ServiceLoader mechanism (This mechanism is also being used to register data sources like parquet, json, jdbc etc.). This allows auto-loading implementations of ExternalClusterManager interface. Currently, when a driver fails, executors exit using system.exit. This does not bode well for cluster managers that would like to reuse the parent process of an executor. Hence, 1. Moving system.exit to a function that can be overriden in subclasses of CoarseGrainedExecutorBackend. 2. Added functionality of killing all the running tasks in an executor. ExternalClusterManagerSuite.scala was added to test this patch. Author: Hemant Bhanawat Closes #11723 from hbhanawat/pluggableScheduler. Conflicts: core/src/main/scala/org/apache/spark/SparkContext.scala core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala core/src/main/scala/org/apache/spark/executor/Executor.scala core/src/main/scala/org/apache/spark/scheduler/ExternalClusterManager.scala core/src/test/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala dev/.rat-excludes --- .../CoarseGrainedExecutorBackend.scala | 16 +++++--- .../org/apache/spark/executor/Executor.scala | 15 +++++++ .../scheduler/ExternalClusterManager.scala | 9 +++-- .../ExternalClusterManagerSuite.scala | 40 +++++-------------- dev/.rat-excludes | 1 + 5 files changed, 42 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 9b62e4b1b7150..8cc65f6e4efa7 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -66,7 +66,8 @@ private[spark] class CoarseGrainedExecutorBackend( case Success(msg) => // Always receive `true`. Just ignore it case Failure(e) => - exitExecutor(1, s"Cannot register with driver: $driverUrl", e, notifyDriver = false) + logError(s"Cannot register with driver: $driverUrl", e) + exitExecutor() }(ThreadUtils.sameThread) } @@ -87,11 +88,13 @@ private[spark] class CoarseGrainedExecutorBackend( } case RegisterExecutorFailed(message) => - exitExecutor(1, "Slave registration failed: " + message) + logError("Slave registration failed: " + message) + exitExecutor() case LaunchTask(data) => if (executor == null) { - exitExecutor(1, "Received LaunchTask command but executor was null") + logError("Received LaunchTask command but executor was null") + exitExecutor() } else { val taskDesc = TaskDescription.decode(data.value) logInfo("Got assigned task " + taskDesc.taskId) @@ -100,7 +103,8 @@ private[spark] class CoarseGrainedExecutorBackend( case KillTask(taskId, _, interruptThread, reason) => if (executor == null) { - exitExecutor(1, "Received KillTask command but executor was null") + logError("Received KillTask command but executor was null") + exitExecutor() } else { executor.killTask(taskId, interruptThread, reason) } @@ -133,8 +137,8 @@ private[spark] class CoarseGrainedExecutorBackend( if (stopping.get()) { logInfo(s"Driver from $remoteAddress disconnected during shutdown") } else if (driver.exists(_.address == remoteAddress)) { - exitExecutor(1, s"Driver $remoteAddress disassociated! Shutting down.", null, - notifyDriver = false) + logError(s"Driver $remoteAddress disassociated! Shutting down.") + exitExecutor() } else { logWarning(s"An unknown ($remoteAddress) driver disconnected.") } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 2c3a8ef74800b..05ec6565dfe25 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -214,6 +214,21 @@ private[spark] class Executor( killTask(t, interruptThread = interruptThread, reason = reason)) } + /** + * Function to kill the running tasks in an executor. + * This can be called by executor back-ends to kill the + * tasks instead of taking the JVM down. + * @param interruptThread whether to interrupt the task thread + */ + def killAllTasks(interruptThread: Boolean) : Unit = { + // kill all the running tasks + for (taskRunner <- runningTasks.values().asScala) { + if (taskRunner != null) { + taskRunner.kill(interruptThread) + } + } + } + def stop(): Unit = { env.metricsSystem.report() heartbeater.shutdown() diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExternalClusterManager.scala b/core/src/main/scala/org/apache/spark/scheduler/ExternalClusterManager.scala index 47f3527a32c01..bd9a3bf67d190 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExternalClusterManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExternalClusterManager.scala @@ -18,11 +18,14 @@ package org.apache.spark.scheduler import org.apache.spark.SparkContext +import org.apache.spark.annotation.DeveloperApi /** + * :: DeveloperApi :: * A cluster manager interface to plugin external scheduler. */ -private[spark] trait ExternalClusterManager { +@DeveloperApi +trait ExternalClusterManager { /** * Check if this cluster manager instance can create scheduler components @@ -42,7 +45,7 @@ private[spark] trait ExternalClusterManager { /** * Create a scheduler backend for the given SparkContext and scheduler. This is - * called after task scheduler is created using `ExternalClusterManager.createTaskScheduler()`. + * called after task scheduler is created using [[ExternalClusterManager.createTaskScheduler()]]. * @param sc SparkContext * @param masterURL the master URL * @param scheduler TaskScheduler that will be used with the scheduler backend. @@ -59,4 +62,4 @@ private[spark] trait ExternalClusterManager { * @param backend SchedulerBackend that works with a TaskScheduler */ def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit -} +} \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index a4e4ea7cd2894..c57a783f2f99f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -20,31 +20,19 @@ package org.apache.spark.scheduler import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.AccumulatorV2 -class ExternalClusterManagerSuite extends SparkFunSuite with LocalSparkContext { +class ExternalClusterManagerSuite extends SparkFunSuite with LocalSparkContext +{ test("launch of backend and scheduler") { val conf = new SparkConf().setMaster("myclusterManager"). setAppName("testcm").set("spark.driver.allowMultipleContexts", "true") sc = new SparkContext(conf) - // check if the scheduler components are created and initialized - sc.schedulerBackend match { - case dummy: DummySchedulerBackend => assert(dummy.initialized) - case other => fail(s"wrong scheduler backend: ${other}") - } - sc.taskScheduler match { - case dummy: DummyTaskScheduler => assert(dummy.initialized) - case other => fail(s"wrong task scheduler: ${other}") - } + // check if the scheduler components are created + assert(sc.schedulerBackend.isInstanceOf[DummySchedulerBackend]) + assert(sc.taskScheduler.isInstanceOf[DummyTaskScheduler]) } } -/** - * Super basic ExternalClusterManager, just to verify ExternalClusterManagers can be configured. - * - * Note that if you want a special ClusterManager for tests, you are probably much more interested - * in [[MockExternalClusterManager]] and the corresponding [[SchedulerIntegrationSuite]] - */ private class DummyExternalClusterManager extends ExternalClusterManager { def canCreate(masterURL: String): Boolean = masterURL == "myclusterManager" @@ -56,15 +44,11 @@ private class DummyExternalClusterManager extends ExternalClusterManager { masterURL: String, scheduler: TaskScheduler): SchedulerBackend = new DummySchedulerBackend() - def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { - scheduler.asInstanceOf[DummyTaskScheduler].initialized = true - backend.asInstanceOf[DummySchedulerBackend].initialized = true - } + def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = {} } private class DummySchedulerBackend extends SchedulerBackend { - var initialized = false def start() {} def stop() {} def reviveOffers() {} @@ -72,22 +56,18 @@ private class DummySchedulerBackend extends SchedulerBackend { } private class DummyTaskScheduler extends TaskScheduler { - var initialized = false - override def schedulingMode: SchedulingMode = SchedulingMode.FIFO - override def rootPool: Pool = new Pool("", schedulingMode, 0, 0) + override def rootPool: Pool = null + override def schedulingMode: SchedulingMode = SchedulingMode.NONE override def start(): Unit = {} override def stop(): Unit = {} override def submitTasks(taskSet: TaskSet): Unit = {} override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = {} - override def killTaskAttempt( - taskId: Long, interruptThread: Boolean, reason: String): Boolean = false override def setDAGScheduler(dagScheduler: DAGScheduler): Unit = {} override def defaultParallelism(): Int = 2 override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} - override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None def executorHeartbeatReceived( execId: String, - accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], + accumUpdates: Array[(Long, Seq[AccumulableInfo])], blockManagerId: BlockManagerId): Boolean = true -} +} \ No newline at end of file diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 607234b4068d0..1043aed01d9a4 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -103,3 +103,4 @@ META-INF/* spark-warehouse structured-streaming/* kafka-source-initial-offset-version-2.1.0.bin +org.apache.spark.scheduler.ExternalClusterManager From 06eca13c64d1ce5e9ad44d556bc7ee21ed40eef9 Mon Sep 17 00:00:00 2001 From: Hemant Bhanawat Date: Wed, 27 Apr 2016 10:59:23 -0700 Subject: [PATCH 002/142] [SPARK-14729][SCHEDULER] Refactored YARN scheduler creation code to use newly added ExternalClusterManager With the addition of ExternalClusterManager(ECM) interface in PR #11723, any cluster manager can now be integrated with Spark. It was suggested in ExternalClusterManager PR that one of the existing cluster managers should start using the new interface to ensure that the API is correct. Ideally, all the existing cluster managers should eventually use the ECM interface but as a first step yarn will now use the ECM interface. This PR refactors YARN code from SparkContext.createTaskScheduler function into YarnClusterManager that implements ECM interface. Since this is refactoring, no new tests has been added. Existing tests have been run. Basic manual testing with YARN was done too. Author: Hemant Bhanawat Closes #12641 from hbhanawat/yarnClusterMgr. Conflicts: core/src/main/scala/org/apache/spark/SparkContext.scala core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala --- .../scala/org/apache/spark/SparkContext.scala | 64 +++++++++++++++++++ .../SparkContextSchedulerCreationSuite.scala | 30 +++++++++ ...che.spark.scheduler.ExternalClusterManager | 1 + .../cluster/YarnClusterManager.scala | 56 ++++++++++++++++ 4 files changed, 151 insertions(+) create mode 100644 yarn/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager create mode 100644 yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3828d4f703247..c44faa5a519ab 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2731,6 +2731,70 @@ object SparkContext extends Logging { } (backend, scheduler) + case "yarn" if deployMode == "cluster" => + val scheduler = try { + val clazz = Utils.classForName("org.apache.spark.scheduler.cluster.YarnClusterScheduler") + val cons = clazz.getConstructor(classOf[SparkContext]) + cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl] + } catch { + // TODO: Enumerate the exact reasons why it can fail + // But irrespective of it, it means we cannot proceed ! + case e: Exception => + throw new SparkException("YARN mode not available ?", e) + } + val backend = try { + val clazz = + Utils.classForName("org.apache.spark.scheduler.cluster.YarnClusterSchedulerBackend") + val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext]) + cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] + } catch { + case e: Exception => + throw new SparkException("YARN mode not available ?", e) + } + scheduler.initialize(backend) + (backend, scheduler) + + case "yarn" if deployMode == "client" => + val scheduler = try { + val clazz = Utils.classForName("org.apache.spark.scheduler.cluster.YarnScheduler") + val cons = clazz.getConstructor(classOf[SparkContext]) + cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl] + + } catch { + case e: Exception => + throw new SparkException("YARN mode not available ?", e) + } + + val backend = try { + val clazz = + Utils.classForName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend") + val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext]) + cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] + } catch { + case e: Exception => + throw new SparkException("YARN mode not available ?", e) + } + + scheduler.initialize(backend) + (backend, scheduler) + + case MESOS_REGEX(mesosUrl) => + MesosNativeLibrary.load() + val scheduler = new TaskSchedulerImpl(sc) + val coarseGrained = sc.conf.getBoolean("spark.mesos.coarse", defaultValue = true) + val backend = if (coarseGrained) { + new CoarseMesosSchedulerBackend(scheduler, sc, mesosUrl, sc.env.securityManager) + } else { + new MesosSchedulerBackend(scheduler, sc, mesosUrl) + } + scheduler.initialize(backend) + (backend, scheduler) + + case zkUrl if zkUrl.startsWith("zk://") => + logWarning("Master URL for a multi-master Mesos cluster managed by ZooKeeper should be " + + "in the form mesos://zk://host:port. Current Master URL will stop working in Spark 2.0.") + createTaskScheduler(sc, "mesos://" + zkUrl, deployMode) + case masterUrl => val cm = getClusterManager(masterUrl) match { case Some(clusterMgr) => clusterMgr diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index f8938dfedee5b..43be36966d922 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -129,4 +129,34 @@ class SparkContextSchedulerCreationSuite case _ => fail() } } + + def testMesos(master: String, expectedClass: Class[_], coarse: Boolean) { + val conf = new SparkConf().set("spark.mesos.coarse", coarse.toString) + try { + val sched = createTaskScheduler(master, "client", conf) + assert(sched.backend.getClass === expectedClass) + } catch { + case e: UnsatisfiedLinkError => + assert(e.getMessage.contains("mesos")) + logWarning("Mesos not available, could not test actual Mesos scheduler creation") + case e: Throwable => fail(e) + } + } + + test("mesos fine-grained") { + testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend], coarse = false) + } + + test("mesos coarse-grained") { + testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend], coarse = true) + } + + test("mesos with zookeeper") { + testMesos("mesos://zk://localhost:1234,localhost:2345", + classOf[MesosSchedulerBackend], coarse = false) + } + + test("mesos with zookeeper and Master URL starting with zk://") { + testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend], coarse = false) + } } diff --git a/yarn/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/yarn/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager new file mode 100644 index 0000000000000..6e8a1ebfc61da --- /dev/null +++ b/yarn/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager @@ -0,0 +1 @@ +org.apache.spark.scheduler.cluster.YarnClusterManager diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala new file mode 100644 index 0000000000000..64cd1bd088001 --- /dev/null +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala @@ -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.spark.scheduler.cluster + +import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} + +/** + * Cluster Manager for creation of Yarn scheduler and backend + */ +private[spark] class YarnClusterManager extends ExternalClusterManager { + + override def canCreate(masterURL: String): Boolean = { + masterURL == "yarn" + } + + override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { + sc.deployMode match { + case "cluster" => new YarnClusterScheduler(sc) + case "client" => new YarnScheduler(sc) + case _ => throw new SparkException(s"Unknown deploy mode '${sc.deployMode}' for Yarn") + } + } + + override def createSchedulerBackend(sc: SparkContext, + masterURL: String, + scheduler: TaskScheduler): SchedulerBackend = { + sc.deployMode match { + case "cluster" => + new YarnClusterSchedulerBackend(scheduler.asInstanceOf[TaskSchedulerImpl], sc) + case "client" => + new YarnClientSchedulerBackend(scheduler.asInstanceOf[TaskSchedulerImpl], sc) + case _ => + throw new SparkException(s"Unknown deploy mode '${sc.deployMode}' for Yarn") + } + } + + override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { + scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend) + } +} From e9f80e697711fb78de85b7ddf03c2cf17f4bd515 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Sat, 21 Nov 2015 22:18:18 +0530 Subject: [PATCH 003/142] [SNAPPYDATA] increasing visibility of SparkContext.activeContext --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c44faa5a519ab..1dcbb87405f3f 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2419,7 +2419,7 @@ object SparkContext extends Logging { * * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK. */ - private val activeContext: AtomicReference[SparkContext] = + private[spark] val activeContext: AtomicReference[SparkContext] = new AtomicReference[SparkContext](null) /** From 009ab91bcfd98d6f94f182d9cff5e0750e7db7b2 Mon Sep 17 00:00:00 2001 From: Neeraj Kumar Date: Wed, 30 Dec 2015 19:17:11 +0530 Subject: [PATCH 004/142] [SNAPPYDATA] add SnappyData's modification headers in updated files --- .../scala/org/apache/spark/SparkContext.scala | 18 ++++++++++++++++++ .../spark/scheduler/TaskSetManager.scala | 18 ++++++++++++++++++ .../apache/spark/sql/AnalysisException.scala | 18 ++++++++++++++++++ .../catalyst/plans/physical/partitioning.scala | 18 ++++++++++++++++++ .../spark/sql/hive/thriftserver/CliSuite.scala | 18 ++++++++++++++++++ .../thriftserver/HiveThriftServer2Suites.scala | 18 ++++++++++++++++++ .../spark/sql/hive/HiveSparkSubmitSuite.scala | 18 ++++++++++++++++++ .../spark/sql/hive/StatisticsSuite.scala | 18 ++++++++++++++++++ 8 files changed, 144 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1dcbb87405f3f..fcc5fce4bb798 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index c3ed11bfe352a..5149cbcdfb3c0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.scheduler diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index 50ee6cd4085ea..741d2aaa3061d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 4d9a9925fe3ff..77b40e6452880 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.catalyst.plans.physical diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 933fd7369380a..521facc027226 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.hive.thriftserver diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index 496f8c82a6c61..b445cc359049f 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.hive.thriftserver diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 21b3e281490cf..fd7f1a535b0a9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.hive diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 3af8af0814bb4..35f13a0626666 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.hive From 8998a02fd97435d3187a7eab7938db16a0052987 Mon Sep 17 00:00:00 2001 From: Soubhik Chakraborty Date: Wed, 13 Jan 2016 03:09:56 +0530 Subject: [PATCH 005/142] [SNAP-404] Address #comment about increasing decimal precision Conflicts: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- .../main/scala/org/apache/spark/sql/types/DecimalType.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala index ef3b67c0d48d0..fbe24995dc51b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala @@ -118,6 +118,10 @@ object DecimalType extends AbstractDataType { val SYSTEM_DEFAULT: DecimalType = DecimalType(MAX_PRECISION, 18) val USER_DEFAULT: DecimalType = DecimalType(10, 0) val MINIMUM_ADJUSTED_SCALE = 6 +// val MAX_PRECISION = 127 +// val MAX_SCALE = 63 +// val SYSTEM_DEFAULT: DecimalType = DecimalType(38, 18) +// val USER_DEFAULT: DecimalType = DecimalType(38, 18) // The decimal types compatible with other numeric types private[sql] val ByteDecimal = DecimalType(3, 0) From cc40bc967be18eab378d1ffddbc0135743631d3c Mon Sep 17 00:00:00 2001 From: SachinJanani Date: Thu, 14 Jul 2016 16:55:06 +0530 Subject: [PATCH 006/142] [SNAP-860] Removed hardcoding of size of Array used for storing DecimalType (#38) Created a jira for -Pspark precheckin SNAP-914 (https://jira.snappydata.io/browse/SNAP-914) --- .../main/scala/org/apache/spark/sql/types/DecimalType.scala | 4 ++-- .../sql/execution/datasources/json/JsonInferSchema.scala | 4 ++-- .../datasources/parquet/ParquetSchemaConverter.scala | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala index fbe24995dc51b..5ad10d9cbf9b9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} * A Decimal that must have fixed precision (the maximum number of digits) and scale (the number * of digits on right side of dot). * - * The precision can be up to 38, scale can also be up to 38 (less or equal to precision). + * The precision can be up to 127, scale can also be up to 127 (less or equal to precision). * * The default precision and scale is (10, 0). * @@ -48,7 +48,7 @@ case class DecimalType(precision: Int, scale: Int) extends FractionalType { } if (precision > DecimalType.MAX_PRECISION) { - throw new AnalysisException(s"DecimalType can only support precision up to 38") + throw new AnalysisException(s"DecimalType can only support precision up to ${DecimalType.MAX_PRECISION}") } // default constructor for Java diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonInferSchema.scala index a270a6451d5dd..1c1b9f0520179 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonInferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonInferSchema.scala @@ -279,8 +279,8 @@ private[sql] object JsonInferSchema { case (t1: DecimalType, t2: DecimalType) => val scale = math.max(t1.scale, t2.scale) val range = math.max(t1.precision - t1.scale, t2.precision - t2.scale) - if (range + scale > 38) { - // DecimalType can't support precision > 38 + if (range + scale > DecimalType.MAX_PRECISION) { + // DecimalType can't support precision > DecimalType.MAX_PRECISION DoubleType } else { DecimalType(range + scale, scale) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala index c61be077d309f..4f64005852110 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala @@ -594,7 +594,7 @@ private[sql] object ParquetSchemaConverter { } // Returns the minimum number of bytes needed to store a decimal with a given `precision`. - val minBytesForPrecision = Array.tabulate[Int](39)(computeMinBytesForPrecision) + val minBytesForPrecision = Array.tabulate[Int](DecimalType.MAX_PRECISION + 1)(computeMinBytesForPrecision) // Max precision of a decimal value stored in `numBytes` bytes def maxPrecisionForBytes(numBytes: Int): Int = { From 79130cb2762d260f80bfbdcf54fba92f5e80000c Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Fri, 29 Jul 2016 10:30:42 +0530 Subject: [PATCH 007/142] [SNAPPYDATA] Try hard to not schedule on others if ExecutorCacheTaskLocation is alive Conflicts: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala --- .../spark/scheduler/TaskSetManager.scala | 37 +++++++++++++++---- 1 file changed, 30 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 5149cbcdfb3c0..3da2116f10d97 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -351,7 +351,9 @@ private[spark] class TaskSetManager( // Check for node-local tasks if (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) { - for (index <- speculatableTasks if canRunOnHost(index)) { + for (index <- speculatableTasks if canRunOnHost(index) && + // don't return executor-local tasks that are still alive + canRunOnExecutor(execId, index)) { val locations = tasks(index).preferredLocations.map(_.host) if (locations.contains(host)) { speculatableTasks -= index @@ -374,7 +376,9 @@ private[spark] class TaskSetManager( // Check for rack-local tasks if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { for (rack <- sched.getRackForHost(host)) { - for (index <- speculatableTasks if canRunOnHost(index)) { + for (index <- speculatableTasks if canRunOnHost(index) + // don't return executor-local tasks that are still alive + if canRunOnExecutor(execId, index)) { val racks = tasks(index).preferredLocations.map(_.host).flatMap(sched.getRackForHost) if (racks.contains(rack)) { speculatableTasks -= index @@ -386,7 +390,9 @@ private[spark] class TaskSetManager( // Check for non-local tasks if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { - for (index <- speculatableTasks if canRunOnHost(index)) { + for (index <- speculatableTasks if canRunOnHost(index) && + // don't return executor-local tasks that are still alive + canRunOnExecutor(execId, index)) { speculatableTasks -= index return Some((index, TaskLocality.ANY)) } @@ -396,6 +402,17 @@ private[spark] class TaskSetManager( None } + private def canRunOnExecutor(execId: String, taskId: Int): Boolean = { + val locations = tasks(taskId).preferredLocations + locations.isEmpty || locations.exists { + case e: ExecutorCacheTaskLocation => execId == e.executorId + case _ => false + } || locations.collectFirst { + case e: ExecutorCacheTaskLocation if sched.isExecutorAlive(e.executorId) + && !executorIsBlacklisted(e.executorId, taskId) => false + }.getOrElse(true) + } + /** * Dequeue a pending task for a given node and return its index and locality level. * Only search for tasks matching the given locality constraint. @@ -410,7 +427,9 @@ private[spark] class TaskSetManager( } if (TaskLocality.isAllowed(maxLocality, TaskLocality.NODE_LOCAL)) { - for (index <- dequeueTaskFromList(execId, host, getPendingTasksForHost(host))) { + for (index <- dequeueTaskFromList(execId, getPendingTasksForHost(host)) + // don't return executor-local tasks that are still alive + if canRunOnExecutor(execId, index)) { return Some((index, TaskLocality.NODE_LOCAL, false)) } } @@ -425,14 +444,18 @@ private[spark] class TaskSetManager( if (TaskLocality.isAllowed(maxLocality, TaskLocality.RACK_LOCAL)) { for { rack <- sched.getRackForHost(host) - index <- dequeueTaskFromList(execId, host, getPendingTasksForRack(rack)) + index <- dequeueTaskFromList(execId, getPendingTasksForRack(rack)) + // don't return executor-local tasks that are still alive + if canRunOnExecutor(execId, index) } { - return Some((index, TaskLocality.RACK_LOCAL, false)) + return Some((index, TaskLocality.RACK_LOCAL, false)) } } if (TaskLocality.isAllowed(maxLocality, TaskLocality.ANY)) { - for (index <- dequeueTaskFromList(execId, host, allPendingTasks)) { + for (index <- dequeueTaskFromList(execId, allPendingTasks) + // don't return executor-local tasks that are still alive + if canRunOnExecutor(execId, index)) { return Some((index, TaskLocality.ANY, false)) } } From 596ac8930efcfcffd0857d3e88b2e3a9e9644980 Mon Sep 17 00:00:00 2001 From: nthanvi Date: Tue, 17 May 2016 14:43:35 +0530 Subject: [PATCH 008/142] [SNAPPYDATA] Use SnappyContext as default SQLContext on shell (#35) Conflicts: python/pyspark/shell.py --- python/pyspark/shell.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index b5fcf7092d93a..516185f10cdce 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -31,6 +31,8 @@ from pyspark import SparkConf from pyspark.context import SparkContext from pyspark.sql import SparkSession, SQLContext +from pyspark.sql.snappy import SnappyContext +from pyspark.storagelevel import StorageLevel if os.environ.get("SPARK_EXECUTOR_URI"): SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"]) @@ -38,6 +40,8 @@ SparkContext._ensure_initialized() try: + sqlContext = SnappyContext(sc) +except py4j.protocol.Py4JError: # Try to access HiveConf, it will raise exception if Hive is not added conf = SparkConf() if conf.get('spark.sql.catalogImplementation', 'hive').lower() == 'hive': From f5bc2d6b4c7e48a3ed001d96cc7b7f25f85914b2 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Tue, 29 Mar 2016 00:06:36 +0530 Subject: [PATCH 009/142] [SNAP-643] Increase visibility of some methods in GenerateUnsafeProjection * increase visibility of complex type write methods (that perform code generation) in GenerateUnsafeProjection to allow using from outside * allow for internal types (ArrayData, MapData, InternalRow) directly in Row->InternalRow conversions in CatalystTypeConverters for complex types Conflicts: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala --- .../spark/sql/catalyst/CatalystTypeConverters.scala | 4 ++++ .../expressions/codegen/GenerateUnsafeProjection.scala | 10 ++++++---- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index 474ec592201d9..6a96aab7ad9dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -170,6 +170,7 @@ object CatalystTypeConverters { convertedIterable += elementConverter.toCatalyst(item) } new GenericArrayData(convertedIterable.toArray) + case a: ArrayData => a } } @@ -206,6 +207,7 @@ object CatalystTypeConverters { scalaValue match { case map: Map[_, _] => ArrayBasedMapData(map, keyFunction, valueFunction) case javaMap: JavaMap[_, _] => ArrayBasedMapData(javaMap, keyFunction, valueFunction) + case m: MapData => m } } @@ -252,6 +254,8 @@ object CatalystTypeConverters { idx += 1 } new GenericInternalRow(ar) + + case row: InternalRow => row } override def toScala(row: InternalRow): Row = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 36ffa8dcdd2b6..396d66ba92542 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -44,7 +44,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro } // TODO: if the nullability of field is correct, we can use it to save null check. - private def writeStructToBuffer( + private[sql] def writeStructToBuffer( ctx: CodegenContext, input: String, fieldTypes: Seq[DataType], @@ -177,7 +177,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro } // TODO: if the nullability of array element is correct, we can use it to save null check. - private def writeArrayToBuffer( + private[sql] def writeArrayToBuffer( ctx: CodegenContext, input: String, elementType: DataType, @@ -213,11 +213,13 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro $arrayWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); """ - case a @ ArrayType(et, _) => + case a @ ArrayType(at, _) => s""" final int $tmpCursor = $bufferHolder.cursor; ${writeArrayToBuffer(ctx, element, et, bufferHolder)} $arrayWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); + $arrayWriter.setOffset($index); + ${writeArrayToBuffer(ctx, element, at, bufferHolder)} """ case m @ MapType(kt, vt, _) => @@ -256,7 +258,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro } // TODO: if the nullability of value element is correct, we can use it to save null check. - private def writeMapToBuffer( + private[sql] def writeMapToBuffer( ctx: CodegenContext, input: String, keyType: DataType, From 3104dd1d7ffbec98c6ede50d11fabc7a5c6e45f3 Mon Sep 17 00:00:00 2001 From: Asif Shahid Date: Wed, 27 Apr 2016 21:54:33 -0700 Subject: [PATCH 010/142] [SNAPPYDATA] Fixing sequence of expression in an option Fixing TakeOrderedAndProject that contains Sequence of Expression in an option, but spark plan expression transformation used to skip it as it does not handle sequence in an Option. Conflicts: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala --- .../spark/sql/catalyst/plans/QueryPlan.scala | 25 ++++++++++++++++--- 1 file changed, 22 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index ddf2cbf2ab911..d3dbb7851b828 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.catalyst.plans @@ -113,8 +131,9 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT } def recursiveTransform(arg: Any): AnyRef = arg match { - case e: Expression => transformExpression(e) - case Some(value) => Some(recursiveTransform(value)) + case e: Expression => transformExpressionUp(e) + case Some(e: Expression) => Some(transformExpressionUp(e)) + case Some(seq: Traversable[_]) => Some(seq.map(recursiveTransform)) case m: Map[_, _] => m case d: DataType => d // Avoid unpacking Structs case seq: Traversable[_] => seq.map(recursiveTransform) @@ -148,7 +167,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT productIterator.flatMap { case e: Expression => e :: Nil - case s: Some[_] => seqToExpressions(s.toSeq) + case Some(seq: Traversable[_] ) => seqToExpressions(seq) case seq: Traversable[_] => seqToExpressions(seq) case other => Nil }.toSeq From 4cf0d5d721797f7101399f6b2b7193ad01713983 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Thu, 28 Jul 2016 21:51:57 +0530 Subject: [PATCH 011/142] [SNAP-931] Use non-secure randomUUID where appropriate (#40) - adding a non-secure version for random UUID adapted from Android UUID.java - use the same for file name in DiskBlockManager, Utils methods, WriteAheadLogBackedBlockRDD Conflicts: core/src/main/scala/org/apache/spark/util/Utils.scala Conflicts: core/src/main/scala/org/apache/spark/util/Utils.scala streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala --- .../spark/storage/DiskBlockManager.scala | 9 ++-- .../apache/spark/storage/StorageUtils.scala | 49 +++++++++++++++++++ .../scala/org/apache/spark/util/Utils.scala | 26 +++++++++- .../rdd/WriteAheadLogBackedBlockRDD.scala | 6 +-- 4 files changed, 79 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index a69bcc9259995..84ee3749cc1f4 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -18,7 +18,6 @@ package org.apache.spark.storage import java.io.{File, IOException} -import java.util.UUID import org.apache.spark.SparkConf import org.apache.spark.executor.ExecutorExitCode @@ -114,18 +113,18 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea /** Produces a unique block id and File suitable for storing local intermediate results. */ def createTempLocalBlock(): (TempLocalBlockId, File) = { - var blockId = new TempLocalBlockId(UUID.randomUUID()) + var blockId = new TempLocalBlockId(StorageUtils.newNonSecureRandomUUID()) while (getFile(blockId).exists()) { - blockId = new TempLocalBlockId(UUID.randomUUID()) + blockId = new TempLocalBlockId(StorageUtils.newNonSecureRandomUUID()) } (blockId, getFile(blockId)) } /** Produces a unique block id and File suitable for storing shuffled intermediate results. */ def createTempShuffleBlock(): (TempShuffleBlockId, File) = { - var blockId = new TempShuffleBlockId(UUID.randomUUID()) + var blockId = new TempShuffleBlockId(StorageUtils.newNonSecureRandomUUID()) while (getFile(blockId).exists()) { - blockId = new TempShuffleBlockId(UUID.randomUUID()) + blockId = new TempShuffleBlockId(StorageUtils.newNonSecureRandomUUID()) } (blockId, getFile(blockId)) } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index e9694fdbca2de..4b0da68f59b4d 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -18,6 +18,7 @@ package org.apache.spark.storage import java.nio.{ByteBuffer, MappedByteBuffer} +import java.util.UUID import scala.collection.Map import scala.collection.mutable @@ -331,4 +332,52 @@ private[spark] object StorageUtils extends Logging { blockLocations } + /** static random number generator for UUIDs */ + private val uuidRnd = new java.util.Random + + /** + * Generate a random UUID for file names etc. Uses non-secure version + * of random number generator to be more efficient given that its not + * critical to have this unique. + * + * Adapted from Android's java.util.UUID source. + */ + final def newNonSecureRandomUUID(): UUID = { + val randomBytes: Array[Byte] = new Array[Byte](16) + uuidRnd.nextBytes(randomBytes) + + var msb = getLong(randomBytes, 0) + var lsb = getLong(randomBytes, 8) + // Set the version field to 4. + msb &= ~(0xfL << 12) + msb |= (4L << 12) + // Set the variant field to 2. Note that the variant field is + // variable-width, so supporting other variants is not just a matter + // of changing the constant 2 below! + lsb &= ~(0x3L << 62) + lsb |= 2L << 62 + new UUID(msb, lsb) + } + + final def getLong(src: Array[Byte], offset: Int): Long = { + var index = offset + var h: Int = (src(index) & 0xff) << 24 + index += 1 + h |= (src(index) & 0xff) << 16 + index += 1 + h |= (src(index) & 0xff) << 8 + index += 1 + h |= (src(index) & 0xff) + index += 1 + + var l = (src(index) & 0xff) << 24 + index += 1 + l |= (src(index) & 0xff) << 16 + index += 1 + l |= (src(index) & 0xff) << 8 + index += 1 + l |= (src(index) & 0xff) + + (h.toLong << 32L) | (l.toLong & 0xffffffffL) + } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 5853302973140..20622010ea60a 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -63,6 +63,8 @@ import org.apache.spark.internal.config._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} +import org.apache.spark.util.logging.RollingFileAppender +import org.apache.spark.storage.StorageUtils /** CallSite represents a place in user code. It can have a short and a long form. */ private[spark] case class CallSite(shortForm: String, longForm: String) @@ -296,7 +298,8 @@ private[spark] object Utils extends Logging { maxAttempts + " attempts!") } try { - dir = new File(root, namePrefix + "-" + UUID.randomUUID.toString) + dir = new File(root, namePrefix + "-" + + StorageUtils.newNonSecureRandomUUID().toString) if (dir.exists() || !dir.mkdirs()) { dir = null } @@ -2589,7 +2592,26 @@ private[spark] object Utils extends Logging { * Returns a path of temporary file which is in the same directory with `path`. */ def tempFileWith(path: File): File = { - new File(path.getAbsolutePath + "." + UUID.randomUUID()) + var temp: File = null + do { + temp = new File(path.getAbsolutePath + "." + + StorageUtils.newNonSecureRandomUUID()) + } while (temp.exists()) + } + + /** + * Returns a path of temporary file which is in the same directory with `path`. + */ + def tempFileWith(parent: String, prefix: String): File = { + var temp: File = null + do { + val name = if (prefix == null) { + StorageUtils.newNonSecureRandomUUID().toString + } else { + prefix + '.' + StorageUtils.newNonSecureRandomUUID().toString + } + temp = new File(parent, name) + } while (temp.exists()) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala index 844760ab61d2e..41f7c60e1cc35 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala @@ -16,9 +16,7 @@ */ package org.apache.spark.streaming.rdd -import java.io.File import java.nio.ByteBuffer -import java.util.UUID import scala.reflect.ClassTag import scala.util.control.NonFatal @@ -135,8 +133,8 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( // FileBasedWriteAheadLog will not create any file or directory at that path. Also, // this dummy directory should not already exist otherwise the WAL will try to recover // past events from the directory and throw errors. - val nonExistentDirectory = new File( - System.getProperty("java.io.tmpdir"), UUID.randomUUID().toString).getAbsolutePath + val nonExistentDirectory = Utils.tempFileWith( + System.getProperty("java.io.tmpdir"), prefix = null).getAbsolutePath writeAheadLog = WriteAheadLogUtils.createLogForReceiver( SparkEnv.get.conf, nonExistentDirectory, hadoopConf) dataRead = writeAheadLog.read(partition.walRecordHandle) From a6d7679a54ad4d741fa7bb320681ebb55a056c81 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Fri, 29 Jul 2016 12:10:09 +0530 Subject: [PATCH 012/142] [SNAPPYDATA] Adding SnappyData modification headers for missing files --- python/pyspark/shell.py | 19 +++++++++++++++++++ .../sql/catalyst/CatalystTypeConverters.scala | 18 ++++++++++++++++++ .../codegen/GenerateUnsafeProjection.scala | 18 ++++++++++++++++++ .../apache/spark/sql/types/DecimalType.scala | 18 ++++++++++++++++++ .../datasources/json/JsonInferSchema.scala | 18 ++++++++++++++++++ .../parquet/ParquetSchemaConverter.scala | 18 ++++++++++++++++++ 6 files changed, 109 insertions(+) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 516185f10cdce..8122533f66cb1 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -15,6 +15,25 @@ # limitations under the License. # +# +# Changes for SnappyData data platform. +# +# Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. +# +# Licensed 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. See accompanying +# LICENSE file. +# + """ An interactive shell. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index 6a96aab7ad9dd..86a625a6c8272 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.catalyst diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 396d66ba92542..d9e9c866aeb5b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.catalyst.expressions.codegen diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala index 5ad10d9cbf9b9..c19067c73955f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.types diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonInferSchema.scala index 1c1b9f0520179..f3f0d7046de06 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonInferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonInferSchema.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.execution.datasources.json diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala index 4f64005852110..6eeaa02388379 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.execution.datasources.parquet From b1dafd55c8eac6fd022c047942d35a487cf92185 Mon Sep 17 00:00:00 2001 From: Hemant Bhanawat Date: Mon, 1 Feb 2016 19:03:27 +0530 Subject: [PATCH 013/142] [SNAPPYDATA] Updated README.md with information on SnappyData's changes --- README.md | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/README.md b/README.md index 1e521a7e7b178..ba75637ba4a8e 100644 --- a/README.md +++ b/README.md @@ -1,3 +1,19 @@ +## SnappyData's extensions to Spark + +- SnappyData collocates Spark executors with its in-memory data store in the same JVM. To achieve this, support for external cluster manager in Spark 2.0 is used to add a SnappyData cluster manager. +- SnappyData's MemoryManager was needed to generate and handle memory events. A property spark.memory.manager is now used to specify a memory manager other than Spark's own. +- To display the consumption of memory in an external embedded store, Spark's storage UI was updated. +- Support for getting length of type (for VARCHAR) was added in the JDBCDialect class. +- For SnappyData, dynamic continous queries on streams would be enabled in future. For that, support for registering DStreams after streaming context has started is added. +- For partitioning, sequence of expressions can be provided. SnappyData adds OrderlessHashPartitioning that does not take into account order of expressions while partitioning. +- Hive client thread-local configuration changed to be instance specific. +- Hive client added support for dropTable and listing tables for all databases. +- RDD partitions with executor specific preferred locations will be forced to be routed to one of those executors if alive. +- An "unsecure" version of random UUID added in DiskBlockManager for temporary file names. +- Added a fix for SPARK-13116. +- Increased visibility of some classes/methods. + + # Apache Spark Spark is a fast and general cluster computing system for Big Data. It provides From 71b22d0f646036efe8cbfb927f15769fa1a71054 Mon Sep 17 00:00:00 2001 From: ahshahid Date: Tue, 7 Jun 2016 14:15:36 -0700 Subject: [PATCH 014/142] [SNAPPYDATA] Optimizations for bootstrap * Added a method to bump up the expr id counter by a given number, so as to reserve the ExprID * Optimizing the Declarative aggregate function to have predictable input buffer aggregte attribute reference by using reservation in the ExprID being generated * Changes to minimize the query plan size for bootstrap and some more optimizations which aids in perf improvement * fixed scala style failures Conflicts: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala --- .../expressions/aggregate/interfaces.scala | 30 +++++++++++++- .../expressions/namedExpressions.scala | 21 +++++++++- .../sql/execution/aggregate/AggUtils.scala | 40 ++++++++++++++++++- .../aggregate/HashAggregateExec.scala | 24 ++++++++++- .../aggregate/SortAggregateExec.scala | 24 ++++++++++- 5 files changed, 131 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index e1d16a2cd38b0..d6d394f5b8ae9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.catalyst.expressions.aggregate @@ -379,8 +397,16 @@ abstract class DeclarativeAggregate /** An expression-based aggregate's bufferSchema is derived from bufferAttributes. */ final override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes) - final lazy val inputAggBufferAttributes: Seq[AttributeReference] = - aggBufferAttributes.map(_.newInstance()) + lazy val inputAggBufferbaseExprID = NamedExpression.allocateExprID(aggBufferAttributes.length) + + /* final lazy val inputAggBufferAttributes: Seq[AttributeReference] = + aggBufferAttributes.map(_.newInstance()) */ + + @transient final lazy val inputAggBufferAttributes: Seq[AttributeReference] = + aggBufferAttributes.zipWithIndex.map { + case ( attr, i) => attr.withExprId( ExprId( inputAggBufferbaseExprID.id + i, + inputAggBufferbaseExprID.jvmId)) + } /** * A helper class for representing an attribute used in merging two diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 8df870468c2ad..8f66478011a84 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.catalyst.expressions @@ -30,7 +48,8 @@ object NamedExpression { private val curId = new java.util.concurrent.atomic.AtomicLong() private[expressions] val jvmId = UUID.randomUUID() def newExprId: ExprId = ExprId(curId.getAndIncrement(), jvmId) - def unapply(expr: NamedExpression): Option[(String, DataType)] = Some((expr.name, expr.dataType)) + def unapply(expr: NamedExpression): Option[(String, DataType)] = Some(expr.name, expr.dataType) + def allocateExprID(quota: Int): ExprId = ExprId(curId.getAndAdd(quota), jvmId) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala index ebbdf1aaa024d..0c3161662e717 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.execution.aggregate @@ -27,6 +45,26 @@ import org.apache.spark.sql.internal.SQLConf * Utility functions used by the query planner to convert our plan to new aggregation code path. */ object AggUtils { + + def planAggregateWithoutPartial( + groupingExpressions: Seq[NamedExpression], + aggregateExpressions: Seq[AggregateExpression], + resultExpressions: Seq[NamedExpression], + child: SparkPlan): Seq[SparkPlan] = { + + val completeAggregateExpressions = aggregateExpressions.map(_.copy(mode = Complete)) + val completeAggregateAttributes = completeAggregateExpressions.map(_.resultAttribute) + SortAggregateExec( + requiredChildDistributionExpressions = Some(groupingExpressions), + groupingExpressions = groupingExpressions, + aggregateExpressions = completeAggregateExpressions, + aggregateAttributes = completeAggregateAttributes, + initialInputBufferOffset = 0, + resultExpressions = resultExpressions, + child = child + ) :: Nil + } + private def createAggregate( requiredChildDistributionExpressions: Option[Seq[Expression]] = None, groupingExpressions: Seq[NamedExpression] = Nil, @@ -44,7 +82,7 @@ object AggUtils { aggregateExpressions = aggregateExpressions, aggregateAttributes = aggregateAttributes, initialInputBufferOffset = initialInputBufferOffset, - resultExpressions = resultExpressions, + __resultExpressions = resultExpressions, child = child) } else { val objectHashEnabled = child.sqlContext.conf.useObjectHashAggregation diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index ce3c68810f3b6..cfdda6a046f80 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.execution.aggregate @@ -42,11 +60,13 @@ case class HashAggregateExec( aggregateExpressions: Seq[AggregateExpression], aggregateAttributes: Seq[Attribute], initialInputBufferOffset: Int, - resultExpressions: Seq[NamedExpression], + __resultExpressions: Seq[NamedExpression], child: SparkPlan) extends UnaryExecNode with CodegenSupport { - private[this] val aggregateBufferAttributes = { + @transient lazy val resultExpressions = __resultExpressions + + @transient lazy private[this] val aggregateBufferAttributes = { aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index fc87de2c52e41..00de22e54ab07 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.execution.aggregate @@ -36,11 +54,13 @@ case class SortAggregateExec( aggregateExpressions: Seq[AggregateExpression], aggregateAttributes: Seq[Attribute], initialInputBufferOffset: Int, - resultExpressions: Seq[NamedExpression], + __resultExpressions: Seq[NamedExpression], child: SparkPlan) extends UnaryExecNode { - private[this] val aggregateBufferAttributes = { + @transient lazy val resultExpressions = __resultExpressions + + @transient lazy private[this] val aggregateBufferAttributes = { aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) } From 490663f76ee47b3c181bd8e80f827e74e54102f2 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Sun, 31 Jul 2016 21:54:37 +0530 Subject: [PATCH 015/142] [SNAPPYDATA] Gradle build scripts and build fixes - Adding gradle build scripts with gradle wrapper invocation for all projects/subprojects - Added product target to pack snappy-spark distribution (like dev/make-distribution.sh) - Changes to make it compatible with top-level SnappyData build - Add SnappyData modification headers to remaining modified files - Fixed compilation and few test issues --- .gitignore | 7 +- assembly/build.gradle | 132 ++++++ build.gradle | 376 ++++++++++++++++++ common/network-common/build.gradle | 26 ++ common/network-shuffle/build.gradle | 30 ++ common/network-yarn/build.gradle | 100 +++++ common/sketch/build.gradle | 28 ++ common/tags/build.gradle | 22 + common/unsafe/build.gradle | 38 ++ core/build.gradle | 171 ++++++++ .../spark/storage/DiskBlockManager.scala | 18 + .../apache/spark/storage/StorageUtils.scala | 18 + .../scala/org/apache/spark/util/Utils.scala | 20 + examples/build.gradle | 43 ++ .../docker-integration-tests/build.gradle | 52 +++ external/flume-sink/build.gradle | 50 +++ external/flume/build.gradle | 40 ++ external/kafka-0-10/build.gradle | 34 ++ external/kafka-0-8/build.gradle | 33 ++ external/spark-ganglia-lgpl/build.gradle | 24 ++ gradle.properties | 5 + gradle/wrapper/gradle-wrapper.jar | Bin 0 -> 53638 bytes gradle/wrapper/gradle-wrapper.properties | 6 + gradlew | 160 ++++++++ gradlew.bat | 90 +++++ graphx/build.gradle | 30 ++ launcher/build.gradle | 39 ++ mllib-local/build.gradle | 29 ++ mllib/build.gradle | 46 +++ repl/build.gradle | 43 ++ settings.gradle | 82 ++++ sql/catalyst/build.gradle | 47 +++ .../apache/spark/sql/AnalysisException.scala | 5 + .../sql/catalyst/expressions/Projection.scala | 2 +- sql/core/build.gradle | 43 ++ sql/hive-thriftserver/build.gradle | 85 ++++ sql/hive/build.gradle | 117 ++++++ .../spark/sql/hive/StatisticsSuite.scala | 18 - streaming/build.gradle | 44 ++ .../rdd/WriteAheadLogBackedBlockRDD.scala | 19 + tools/build.gradle | 25 ++ yarn/build.gradle | 137 +++++++ 42 files changed, 2312 insertions(+), 22 deletions(-) create mode 100644 assembly/build.gradle create mode 100644 build.gradle create mode 100644 common/network-common/build.gradle create mode 100644 common/network-shuffle/build.gradle create mode 100644 common/network-yarn/build.gradle create mode 100644 common/sketch/build.gradle create mode 100644 common/tags/build.gradle create mode 100644 common/unsafe/build.gradle create mode 100644 core/build.gradle create mode 100644 examples/build.gradle create mode 100644 external/docker-integration-tests/build.gradle create mode 100644 external/flume-sink/build.gradle create mode 100644 external/flume/build.gradle create mode 100644 external/kafka-0-10/build.gradle create mode 100644 external/kafka-0-8/build.gradle create mode 100644 external/spark-ganglia-lgpl/build.gradle create mode 100644 gradle.properties create mode 100644 gradle/wrapper/gradle-wrapper.jar create mode 100644 gradle/wrapper/gradle-wrapper.properties create mode 100755 gradlew create mode 100644 gradlew.bat create mode 100644 graphx/build.gradle create mode 100644 launcher/build.gradle create mode 100644 mllib-local/build.gradle create mode 100644 mllib/build.gradle create mode 100644 repl/build.gradle create mode 100644 settings.gradle create mode 100644 sql/catalyst/build.gradle create mode 100644 sql/core/build.gradle create mode 100644 sql/hive-thriftserver/build.gradle create mode 100644 sql/hive/build.gradle create mode 100644 streaming/build.gradle create mode 100644 tools/build.gradle create mode 100644 yarn/build.gradle diff --git a/.gitignore b/.gitignore index 903297db96901..6e8668c46a350 100644 --- a/.gitignore +++ b/.gitignore @@ -71,7 +71,7 @@ spark-*-bin-*.tgz spark-tests.log src_managed/ streaming-tests.log -target/ +build-artifacts/ unit-tests.log work/ @@ -86,8 +86,9 @@ spark-warehouse/ # For R session data .RData .RHistory -.Rhistory *.Rproj *.Rproj.* - .Rproj.user + +# gradle specific +.gradle/ diff --git a/assembly/build.gradle b/assembly/build.gradle new file mode 100644 index 0000000000000..5a0f18c45e3f6 --- /dev/null +++ b/assembly/build.gradle @@ -0,0 +1,132 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project Assembly' + +dependencies { + compile project(subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-catalyst_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-sql_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-hive_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-hive-thriftserver_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-repl_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-streaming_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-streaming-kafka-0.8_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-streaming-kafka-0.10_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-yarn_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-mllib_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-graphx_' + scalaBinaryVersion) + if (rootProject.hasProperty('ganglia')) { + compile project(subprojectBase + 'snappy-spark-ganglia-lgpl_' + scalaBinaryVersion) + } +} + +def cleanProduct() { + delete "${sparkProjectRootDir}/python/lib/pyspark.zip" + delete snappyProductDir +} +clean.doLast { + cleanProduct() +} + +task product(type: Zip) { + def examplesProject = project(subprojectBase + 'snappy-spark-examples_' + scalaBinaryVersion) + String yarnShuffleProject = subprojectBase + 'snappy-spark-network-yarn_' + scalaBinaryVersion + dependsOn jar, examplesProject.jar, "${yarnShuffleProject}:shadowJar" + // create python zip + destinationDir = file("${snappyProductDir}/python/lib") + archiveName = 'pyspark.zip' + from("${sparkProjectRootDir}/python") { + include 'pyspark/**/*' + } + + doFirst { + cleanProduct() + } + doLast { + // copy all runtime dependencies (skip for top-level snappydata builds) + if (rootProject.name == 'snappy-spark') { + copy { + from(configurations.runtime) { + // exclude antlr4 explicitly (runtime is still included) + // that gets pulled by antlr gradle plugin + exclude '**antlr4-4*.jar' + // exclude scalatest included by spark-tags + exclude '**scalatest*.jar' + } + into "${snappyProductDir}/jars" + } + } + // copy scripts, data and other files that are part of distribution + copy { + from(sparkProjectRootDir) { + include 'bin/**' + include 'sbin/**' + include 'conf/**' + include 'data/**' + include 'licenses/**' + include 'python/**' + include 'examples/src/**' + } + into snappyProductDir + } + def sparkR = 'sparkProjectRootDir/R/lib/SparkR' + if (file(sparkR).exists()) { + copy { + from sparkR + into "${snappyProductDir}/R/lib" + } + } + + // copy yarn shuffle shadow jar + copy { + from "${project(yarnShuffleProject).buildDir}/jars" + into "${snappyProductDir}/yarn" + } + // copy examples jars + copy { + from "${examplesProject.buildDir}/jars" + into "${snappyProductDir}/examples/jars" + } + // create RELEASE file, copy README etc for top-level snappy-spark project + if (rootProject.name == 'snappy-spark') { + copy { + from(sparkProjectRootDir) { + include 'LICENSE' + include 'NOTICE' + include 'README.md' + } + into snappyProductDir + } + def releaseFile = file("${snappyProductDir}/RELEASE") + String buildFlags = "" + if (rootProject.hasProperty('docker')) { + buildFlags += ' -Pdocker' + } + if (rootProject.hasProperty('ganglia')) { + buildFlags += ' -Pganglia' + } + String gitRevision = "${gitCmd} rev-parse --short HEAD".execute().text.trim() + if (gitRevision.length() > 0) { + gitRevision = " (git revision ${gitRevision})" + } + + releaseFile.append("Spark ${version}${gitRevision} built for Hadoop ${hadoopVersion}\n") + releaseFile.append("Build flags:${buildFlags}\n") + } + } +} diff --git a/build.gradle b/build.gradle new file mode 100644 index 0000000000000..c235dcf712cc4 --- /dev/null +++ b/build.gradle @@ -0,0 +1,376 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +apply plugin: 'wrapper' + +// TODO: profiles and allow changing hadoopVersion + +buildscript { + repositories { + maven { url 'https://plugins.gradle.org/m2' } + mavenCentral() + } + dependencies { + classpath 'io.snappydata:gradle-scalatest:0.13-1' + classpath 'org.github.ngbinh.scalastyle:gradle-scalastyle-plugin_2.11:0.8.2' + } +} + +description = 'Spark Project' + +allprojects { + // We want to see all test results. This is equivalent to setting --continue + // on the command line. + gradle.startParameter.continueOnFailure = true + + repositories { + mavenCentral() + maven { url 'http://repository.apache.org/snapshots' } + } + + apply plugin: 'idea' + + group = 'io.snappydata' + version = '2.0.1-1' + + ext { + scalaBinaryVersion = '2.11' + scalaVersion = scalaBinaryVersion + '.8' + hadoopVersion = '2.7.2' + protobufVersion = '2.6.1' + jerseyVersion = '2.22.2' + sunJerseyVersion = '1.19.1' + jettyVersion = '9.2.16.v20160414' + log4jVersion = '1.2.17' + slf4jVersion = '1.7.21' + junitVersion = '4.12' + javaxServletVersion = '3.1.0' + guavaVersion = '14.0.1' + hiveVersion = '1.2.1.spark2' + chillVersion = '0.8.0' + nettyVersion = '3.8.0.Final' + nettyAllVersion = '4.0.29.Final' + derbyVersion = '10.12.1.1' + httpClientVersion = '4.5.2' + httpCoreVersion = '4.4.4' + fasterXmlVersion = '2.6.5' + snappyJavaVersion = '1.1.2.6' + parquetVersion = '1.7.0' + hiveParquetVersion = '1.6.0' + metricsVersion = '3.1.2' + thriftVersion = '0.9.3' + antlrVersion = '4.5.3' + jpamVersion = '1.1' + seleniumVersion = '2.52.0' + curatorVersion = '2.7.1' + commonsCodecVersion = '1.10' + avroVersion = '1.7.7' + jsr305Version = '3.0.1' + scalatestVersion = '2.2.6' + pegdownVersion = '1.6.0' + + shadePackageName = 'org.spark_project' + } + + // default output directory like in sbt/maven + buildDir = 'build-artifacts/scala-' + scalaBinaryVersion + + ext { + if (rootProject.name == 'snappy-spark') { + subprojectBase = ':' + sparkProjectRoot = ':' + sparkProjectRootDir = project(':').projectDir + testResultsBase = "${rootProject.buildDir}/tests" + gitCmd = "git --git-dir=${rootDir}/.git --work-tree=${rootDir}" + } else { + subprojectBase = ':snappy-spark:' + sparkProjectRoot = ':snappy-spark' + sparkProjectRootDir = project(':snappy-spark').projectDir + testResultsBase = "${rootProject.buildDir}/tests/spark" + gitCmd = "git --git-dir=${project(sparkProjectRoot).projectDir}/.git --work-tree=${project(sparkProjectRoot).projectDir}" + } + snappyProductDir = "${rootProject.buildDir}/snappy" + } +} + +def getStackTrace(def t) { + java.io.StringWriter sw = new java.io.StringWriter() + java.io.PrintWriter pw = new java.io.PrintWriter(sw) + org.codehaus.groovy.runtime.StackTraceUtils.sanitize(t).printStackTrace(pw) + return sw.toString() +} + +task cleanSparkScalaTest << { + def workingDir = "${testResultsBase}/scalatest" + delete workingDir + file(workingDir).mkdirs() +} +task cleanSparkJUnit << { + def workingDir = "${testResultsBase}/junit" + delete workingDir + file(workingDir).mkdirs() +} + +subprojects { + apply plugin: 'scala' + apply plugin: 'maven' + apply plugin: 'scalaStyle' + + // apply compiler options + compileJava.options.encoding = 'UTF-8' + compileJava.options.compilerArgs << '-Xlint:all,-serial,-path,-deprecation' + // compileScala.scalaCompileOptions.optimize = true + compileScala.options.encoding = 'UTF-8' + + javadoc.options.charSet = 'UTF-8' + + scalaStyle { + configLocation = "${sparkProjectRootDir}/scalastyle-config.xml" + inputEncoding = 'UTF-8' + outputEncoding = 'UTF-8' + outputFile = "${buildDir}/scalastyle-output.xml" + includeTestSourceDirectory = false + source = 'src/main/scala' + testSource = 'src/test/scala' + failOnViolation = true + failOnWarning = false + } + + configurations { + runtimeJar { + description 'a dependency to include additional jars at runtime' + visible true + } + } + + // when invoking from snappydata, below are already defined at top-level + if (rootProject.name == 'snappy-spark') { + task packageSources(type: Jar, dependsOn: classes) { + classifier = 'sources' + from sourceSets.main.allSource + } + + configurations { + provided { + description 'a dependency that is provided externally at runtime' + visible true + } + testOutput { + extendsFrom testCompile + description 'a dependency that exposes test artifacts' + } + } + + task packageTests(type: Jar, dependsOn: testClasses) { + description 'Assembles a jar archive of test classes.' + from sourceSets.test.output.classesDir + classifier = 'tests' + } + artifacts { + testOutput packageTests + } + + idea { + module { + scopes.PROVIDED.plus += [ configurations.provided ] + } + } + + sourceSets { + main.compileClasspath += configurations.provided + main.runtimeClasspath -= configurations.provided + test.compileClasspath += configurations.provided + test.runtimeClasspath += configurations.provided + } + + javadoc.classpath += configurations.provided + } + task packageScalaDocs(type: Jar, dependsOn: scaladoc) { + classifier = 'javadoc' + from scaladoc + } + if (rootProject.hasProperty('enablePublish')) { + artifacts { + archives packageScalaDocs, packageSources + } + } + + // fix scala+java mix to all use compileScala which use correct dependency order + sourceSets.main.scala.srcDir 'src/main/java' + sourceSets.main.java.srcDirs = [] + + dependencies { + // This is a dummy dependency that is used along with the shading plug-in + // to create effective poms on publishing (see SPARK-3812). + //compile group: 'org.spark-project.spark', name: 'unused', version: '1.0.0' + compile 'org.scala-lang:scala-library:' + scalaVersion + compile 'org.scala-lang:scala-reflect:' + scalaVersion + + compile group: 'log4j', name:'log4j', version: log4jVersion + compile 'org.slf4j:slf4j-api:' + slf4jVersion + compile 'org.slf4j:slf4j-log4j12:' + slf4jVersion + + testCompile "junit:junit:${junitVersion}" + testCompile "org.scalatest:scalatest_${scalaBinaryVersion}:${scalatestVersion}" + testCompile 'org.mockito:mockito-core:1.10.19' + testCompile 'org.scalacheck:scalacheck_' + scalaBinaryVersion + ':1.12.5' + testCompile 'com.novocode:junit-interface:0.11' + + testRuntime "org.pegdown:pegdown:${pegdownVersion}" + } + + if (rootProject.name == 'snappy-spark') { + task scalaTest(type: Test) { + actions = [ new com.github.maiflai.ScalaTestAction() ] + + List suites = [] + extensions.add(com.github.maiflai.ScalaTestAction.SUITES, suites) + extensions.add('suite', { String name -> suites.add(name) } ) + extensions.add('suites', { String... name -> suites.addAll(name) } ) + + def result = new StringBuilder() + extensions.add(com.github.maiflai.ScalaTestAction.TESTRESULT, result) + extensions.add('testResult', { String name -> result.append(name) } ) + + def output = new StringBuilder() + extensions.add(com.github.maiflai.ScalaTestAction.TESTOUTPUT, output) + extensions.add('testOutput', { String name -> output.append(name) } ) + + def errorOutput = new StringBuilder() + extensions.add(com.github.maiflai.ScalaTestAction.TESTERROR, errorOutput) + extensions.add('testError', { String name -> errorOutput.append(name) } ) + + // running a single scala suite + if (rootProject.hasProperty('singleSuite')) { + suite singleSuite + } + } + } + scalaTest { + // top-level default is single process run since scalatest does not + // spawn separate JVMs + maxParallelForks = 1 + systemProperties 'test.src.tables': '__not_used__' + + workingDir = "${testResultsBase}/scalatest" + + testResult '/dev/tty' + testOutput "${workingDir}/output.txt" + testError "${workingDir}/error.txt" + binResultsDir = file("${workingDir}/binary/${project.name}") + reports.html.destination = file("${workingDir}/html/${project.name}") + reports.junitXml.destination = file(workingDir) + } + test { + jvmArgs '-Xss4096k' + maxParallelForks = (2 * Runtime.getRuntime().availableProcessors()) + systemProperties 'spark.master.rest.enabled': 'false', + 'test.src.tables': 'src' + + workingDir = "${testResultsBase}/junit" + + binResultsDir = file("${workingDir}/binary/${project.name}") + reports.html.destination = file("${workingDir}/html/${project.name}") + reports.junitXml.destination = file(workingDir) + } + // need to do below after graph is ready else it will give an error about + // runtimeClaspath being set after being finalized + gradle.taskGraph.whenReady({ graph -> + tasks.withType(Test).each { test -> + test.configure { + onlyIf { ! Boolean.getBoolean('skip.tests') } + + jvmArgs '-XX:+HeapDumpOnOutOfMemoryError', '-XX:MaxPermSize=512m', '-XX:ReservedCodeCacheSize=512m' + maxHeapSize '3g' + // disable assertions for hive tests as in Spark's pom.xml because HiveCompatibilitySuite currently fails (SPARK-4814) + if (test.project.name.contains('snappy-spark-hive_')) { + jvmArgs '-da' + maxParallelForks = 1 + } else { + jvmArgs '-ea' + } + environment 'SPARK_DIST_CLASSPATH': "${sourceSets.test.runtimeClasspath.asPath}", + 'SPARK_PREPEND_CLASSES': '1', + 'SPARK_SCALA_VERSION': scalaBinaryVersion, + 'SPARK_TESTING': '1', + 'JAVA_HOME': System.getProperty('java.home') + systemProperties 'log4j.configuration': "file:${projectDir}/src/test/resources/log4j.properties", + 'derby.system.durability': 'test', + 'java.awt.headless': 'true', + 'java.io.tmpdir': "${rootProject.buildDir}/tmp", + 'spark.test.home': snappyProductDir, + 'spark.project.home': "${project(sparkProjectRoot).projectDir}", + 'spark.testing': '1', + 'spark.ui.enabled': 'false', + 'spark.ui.showConsoleProgress': 'false', + 'spark.driver.allowMultipleContexts': 'true', + 'spark.unsafe.exceptionOnMemoryLeak': 'true' + + testLogging.exceptionFormat = 'full' + + if (rootProject.name == 'snappy-spark') { + def eol = System.getProperty('line.separator') + beforeTest { desc -> + def now = new Date().format('yyyy-MM-dd HH:mm:ss.SSS Z') + def progress = new File(workingDir, "progress.txt") + def output = new File(workingDir, "output.txt") + progress << "$now Starting test $desc.className $desc.name$eol" + output << "${now} STARTING TEST ${desc.className} ${desc.name}${eol}${eol}" + } + onOutput { desc, event -> + def output = new File(workingDir, "output.txt") + output << event.message + } + afterTest { desc, result -> + def now = new Date().format('yyyy-MM-dd HH:mm:ss.SSS Z') + def progress = new File(workingDir, "progress.txt") + def output = new File(workingDir, "output.txt") + progress << "${now} Completed test ${desc.className} ${desc.name} with result: ${result.resultType}${eol}" + output << "${eol}${now} COMPLETED TEST ${desc.className} ${desc.name} with result: ${result.resultType}${eol}${eol}" + result.exceptions.each { t -> + progress << " EXCEPTION: ${getStackTrace(t)}${eol}" + output << "${getStackTrace(t)}${eol}" + } + } + } + } + } + }) + test.dependsOn subprojectBase + 'cleanSparkJUnit' + scalaTest.dependsOn subprojectBase + 'cleanSparkScalaTest' + check.dependsOn scalaTest + if (rootProject.name == 'snappy-spark') { + check.dependsOn "${subprojectBase}snappy-spark-assembly_${scalaBinaryVersion}:product" + } +} + +task generateSources { + dependsOn subprojectBase + 'snappy-spark-catalyst_' + scalaBinaryVersion + ':generateGrammarSource' + dependsOn subprojectBase + 'snappy-spark-streaming-flume-sink_' + scalaBinaryVersion + ':generateAvroJava' +} + +if (rootProject.name == 'snappy-spark') { + task scalaStyle { + dependsOn subprojects.scalaStyle + } + task check { + dependsOn subprojects.check + } +} else { + scalaStyle.dependsOn subprojects.scalaStyle + check.dependsOn subprojects.check +} diff --git a/common/network-common/build.gradle b/common/network-common/build.gradle new file mode 100644 index 0000000000000..63c23210f801e --- /dev/null +++ b/common/network-common/build.gradle @@ -0,0 +1,26 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project Networking' + +dependencies { + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + + compile group: 'io.netty', name: 'netty-all', version: nettyAllVersion + compile group: 'com.google.code.findbugs', name: 'jsr305', version: jsr305Version + compile group: 'com.google.guava', name: 'guava', version: guavaVersion +} diff --git a/common/network-shuffle/build.gradle b/common/network-shuffle/build.gradle new file mode 100644 index 0000000000000..0ffbc3414ad14 --- /dev/null +++ b/common/network-shuffle/build.gradle @@ -0,0 +1,30 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project Shuffle Streaming Service' + +dependencies { + compile project(subprojectBase + 'snappy-spark-network-common_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + + compile group: 'org.fusesource.leveldbjni', name: 'leveldbjni-all', version: '1.8' + compile group: 'com.fasterxml.jackson.core', name: 'jackson-databind', version: fasterXmlVersion + compile group: 'com.fasterxml.jackson.core', name: 'jackson-annotations', version: fasterXmlVersion + compile group: 'com.google.guava', name: 'guava', version: guavaVersion + + testCompile project(path: subprojectBase + 'snappy-spark-network-common_' + scalaBinaryVersion, configuration: 'testOutput') +} diff --git a/common/network-yarn/build.gradle b/common/network-yarn/build.gradle new file mode 100644 index 0000000000000..323463acad43c --- /dev/null +++ b/common/network-yarn/build.gradle @@ -0,0 +1,100 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +plugins { + id 'com.github.johnrengelman.shadow' version '1.2.3' +} + +description = 'Spark Project YARN Shuffle Service' + +dependencies { + compile project(subprojectBase + 'snappy-spark-network-shuffle_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + + compile group: 'io.netty', name: 'netty-all', version: nettyAllVersion + provided (group: 'org.apache.hadoop', name: 'hadoop-client', version: hadoopVersion) { + exclude(group: 'asm', module: 'asm') + exclude(group: 'org.codehaus.jackson', module: 'jackson-core-asl') + exclude(group: 'org.codehaus.jackson', module: 'jackson-mapper-asl') + exclude(group: 'org.ow2.asm', module: 'asm') + exclude(group: 'org.jboss.netty', module: 'netty') + exclude(group: 'commons-logging', module: 'commons-logging') + exclude(group: 'org.mockito', module: 'mockito-all') + exclude(group: 'org.mortbay.jetty', module: 'servlet-api-2.5') + exclude(group: 'javax.servlet', module: 'servlet-api') + exclude(group: 'junit', module: 'junit') + exclude(group: 'com.google.guava', module: 'guava') + exclude(group: 'com.sun.jersey') + exclude(group: 'com.sun.jersey.jersey-test-framework') + exclude(group: 'com.sun.jersey.contribs') + } + + /* + runtimeJar project(subprojectBase + 'snappy-spark-network-common_' + scalaBinaryVersion) + runtimeJar project(subprojectBase + 'snappy-spark-network-shuffle_' + scalaBinaryVersion) + runtimeJar group: 'io.netty', name: 'netty-all', version: nettyAllVersion + runtimeJar group: 'com.fasterxml.jackson.core', name: 'jackson-databind', version: fasterXmlVersion + runtimeJar group: 'com.fasterxml.jackson.core', name: 'jackson-annotations', version: fasterXmlVersion + */ +} + +shadowJar { + baseName 'snappy-spark' + classifier 'yarn-shuffle' + + mergeServiceFiles { + exclude 'META-INF/*.SF' + exclude 'META-INF/*.DSA' + exclude 'META-INF/*.RSA' + } + + dependencies { + exclude(dependency('org.scala-lang:.*')) + exclude(dependency('org.scala-lang.modules:.*')) + exclude(dependency('org.slf4j:.*')) + exclude(dependency('log4j:.*')) + exclude(dependency('org.scalatest:.*')) + } + //configurations = [ project.configurations.runtimeJar ] + + relocate 'io.netty', "${shadePackageName}.io.netty" + relocate 'com.fasterxml.jackson', "${shadePackageName}.com.fasterxml.jackson" + relocate 'com.google.common', "${shadePackageName}.guava" + + String createdBy = "" + if (rootProject.hasProperty('enablePublish')) { + createdBy = "SnappyData Build Team" + } else { + createdBy = System.getProperty("user.name") + } + manifest { + attributes( + "Manifest-Version" : "1.0", + "Created-By" : createdBy, + "Title" : project.name, + "Version" : version, + "Vendor" : "SnappyData, Inc." + ) + } + + doLast { + copy { + from outputs + into "${buildDir}/jars" + } + } +} diff --git a/common/sketch/build.gradle b/common/sketch/build.gradle new file mode 100644 index 0000000000000..a5e5efff08b5a --- /dev/null +++ b/common/sketch/build.gradle @@ -0,0 +1,28 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project Sketch' + +dependencies { + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) +} + +tasks.withType(JavaCompile) { + options.compilerArgs << '-XDignore.symbol.file' + options.fork = true + options.forkOptions.executable = 'javac' +} diff --git a/common/tags/build.gradle b/common/tags/build.gradle new file mode 100644 index 0000000000000..e272cfbaa6382 --- /dev/null +++ b/common/tags/build.gradle @@ -0,0 +1,22 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project Tags' + +dependencies { + compile "org.scalatest:scalatest_${scalaBinaryVersion}:${scalatestVersion}" +} diff --git a/common/unsafe/build.gradle b/common/unsafe/build.gradle new file mode 100644 index 0000000000000..69d29942f5f1c --- /dev/null +++ b/common/unsafe/build.gradle @@ -0,0 +1,38 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project Unsafe' + +dependencies { + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + + compile group: 'com.twitter', name: 'chill_' + scalaBinaryVersion, version: chillVersion + compile group: 'com.google.code.findbugs', name: 'jsr305', version: jsr305Version + compile group: 'com.google.guava', name: 'guava', version: guavaVersion + + testCompile group: 'org.apache.commons', name: 'commons-lang3', version: '3.3.2' +} + +// reset the srcDirs to allow javac compilation with specific args below +sourceSets.main.scala.srcDirs = [ 'src/main/scala' ] +sourceSets.main.java.srcDirs = [ 'src/main/java' ] + +tasks.withType(JavaCompile) { + options.compilerArgs << '-XDignore.symbol.file' + options.fork = true + options.forkOptions.executable = 'javac' +} diff --git a/core/build.gradle b/core/build.gradle new file mode 100644 index 0000000000000..e1b52bfec0799 --- /dev/null +++ b/core/build.gradle @@ -0,0 +1,171 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project Core' + +dependencies { + compile project(subprojectBase + 'snappy-spark-launcher_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-network-common_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-network-shuffle_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-unsafe_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + + compile(group: 'org.apache.avro', name: 'avro-ipc', version: avroVersion) { + exclude(group: 'io.netty', module: 'netty') + exclude(group: 'org.mortbay.jetty', module: 'jetty') + exclude(group: 'org.mortbay.jetty', module: 'jetty-util') + exclude(group: 'org.mortbay.jetty', module: 'servlet-api') + exclude(group: 'org.apache.velocity', module: 'velocity') + } + compile(group: 'org.apache.avro', name: 'avro-mapred', version: avroVersion, classifier: 'hadoop2') { + exclude(group: 'io.netty', module: 'netty') + exclude(group: 'org.mortbay.jetty', module: 'jetty') + exclude(group: 'org.mortbay.jetty', module: 'jetty-util') + exclude(group: 'org.mortbay.jetty', module: 'servlet-api') + exclude(group: 'org.apache.velocity', module: 'velocity') + exclude(group: 'org.apache.avro', module: 'avro-ipc') + } + compile group: 'com.google.guava', name: 'guava', version: guavaVersion + compile group: 'com.twitter', name: 'chill_' + scalaBinaryVersion, version: chillVersion + compile group: 'com.twitter', name: 'chill-java', version: chillVersion + compile group: 'org.apache.xbean', name: 'xbean-asm5-shaded', version: '4.4' + // explicitly include netty from akka-remote to not let zookeeper override it + compile group: 'io.netty', name: 'netty', version: nettyVersion + // explicitly exclude old netty from zookeeper + compile(group: 'org.apache.zookeeper', name: 'zookeeper', version: '3.4.8') { + exclude(group: 'org.jboss.netty', module: 'netty') + exclude(group: 'jline', module: 'jline') + } + compile group: 'com.google.protobuf', name: 'protobuf-java', version: protobufVersion + compile(group: 'org.apache.hadoop', name: 'hadoop-client', version: hadoopVersion) { + exclude(group: 'asm', module: 'asm') + exclude(group: 'org.codehaus.jackson', module: 'jackson-mapper-asl') + exclude(group: 'org.ow2.asm', module: 'asm') + exclude(group: 'org.apache.zookeeper', module: 'zookeeper') + exclude(group: 'org.jboss.netty', module: 'netty') + exclude(group: 'commons-logging', module: 'commons-logging') + exclude(group: 'org.mockito', module: 'mockito-all') + exclude(group: 'org.mortbay.jetty', module: 'servlet-api-2.5') + exclude(group: 'javax.servlet', module: 'servlet-api') + exclude(group: 'junit', module: 'junit') + exclude(group: 'com.google.guava', module: 'guava') + exclude(group: 'com.sun.jersey') + exclude(group: 'com.sun.jersey.jersey-test-framework') + exclude(group: 'com.sun.jersey.contribs') + exclude(group: 'com.google.protobuf', module: 'protobuf-java') + } + compile(group: 'net.java.dev.jets3t', name: 'jets3t', version: '0.9.3') { + exclude(group: 'commons-logging', module: 'commons-logging') + } + compile(group: 'org.apache.curator', name: 'curator-recipes', version: curatorVersion) { + exclude(group: 'org.apache.zookeeper', module: 'zookeeper') + exclude(group: 'org.jboss.netty', module: 'netty') + exclude(group: 'jline', module: 'jline') + exclude(group: 'com.google.guava', module: 'guava') + } + + compile 'org.scala-lang:scalap:' + scalaVersion + compile group: 'org.apache.xbean', name: 'xbean-asm5-shaded' , version: '4.4' + compile group: 'org.roaringbitmap', name: 'RoaringBitmap' , version: '0.5.11' + + compile group: 'org.eclipse.jetty', name: 'jetty-server', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-plus', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-util', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-http', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-servlet', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-servlets', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-security', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-continuation', version: jettyVersion + compile group: 'javax.servlet', name: 'javax.servlet-api', version: javaxServletVersion + compile group: 'org.apache.commons', name: 'commons-lang3', version: '3.3.2' + compile group: 'org.apache.commons', name: 'commons-math3', version: '3.4.1' + compile group: 'com.google.code.findbugs', name: 'jsr305', version: jsr305Version + compile group: 'org.slf4j', name: 'jul-to-slf4j', version: slf4jVersion + compile group: 'org.slf4j', name: 'jcl-over-slf4j', version: slf4jVersion + compile group: 'com.ning', name: 'compress-lzf', version: '1.0.3' + compile group: 'org.xerial.snappy', name: 'snappy-java', version: snappyJavaVersion + compile group: 'net.jpountz.lz4', name: 'lz4', version: '1.3.0' + compile group: 'commons-net', name: 'commons-net', version: '2.2' + compile group: 'org.json4s', name: 'json4s-jackson_' + scalaBinaryVersion, version: '3.2.11' + compile group: 'org.glassfish.jersey.core', name: 'jersey-client', version: jerseyVersion + compile group: 'org.glassfish.jersey.core', name: 'jersey-common', version: jerseyVersion + compile group: 'org.glassfish.jersey.core', name: 'jersey-server', version: jerseyVersion + compile group: 'org.glassfish.jersey.containers', name: 'jersey-container-servlet', version: jerseyVersion + compile group: 'org.glassfish.jersey.containers', name: 'jersey-container-servlet-core', version: jerseyVersion + compile(group: 'org.apache.mesos', name: 'mesos', version: '0.21.1', classifier: 'shaded-protobuf') { + exclude(group: 'com.google.protobuf', module: 'protobuf-java') + } + compile group: 'io.netty', name: 'netty-all', version: nettyAllVersion + compile(group: 'com.clearspring.analytics', name: 'stream', version: '2.7.0') { + exclude(group: 'it.unimi.dsi', module: 'fastutil') + } + compile group: 'io.dropwizard.metrics', name: 'metrics-core', version: metricsVersion + compile group: 'io.dropwizard.metrics', name: 'metrics-jvm', version: metricsVersion + compile group: 'io.dropwizard.metrics', name: 'metrics-json', version: metricsVersion + compile group: 'io.dropwizard.metrics', name: 'metrics-graphite', version: metricsVersion + compile group: 'com.fasterxml.jackson.core', name: 'jackson-databind', version: fasterXmlVersion + compile(group: 'com.fasterxml.jackson.module', name: 'jackson-module-scala_' + scalaBinaryVersion, version: fasterXmlVersion) { + exclude(group: 'com.google.guava', module: 'guava') + } + compile group: 'org.apache.ivy', name: 'ivy', version: '2.4.0' + compile group: 'oro', name: 'oro', version: '2.0.8' + compile(group: 'net.razorvine', name: 'pyrolite', version: '4.9') { + exclude(group: 'net.razorvine', module: 'serpent') + } + compile group: 'net.sf.py4j', name: 'py4j', version: '0.10.1' + + testCompile group: 'org.apache.avro', name: 'avro-ipc', version: avroVersion, classifier: 'tests' + testCompile "org.apache.derby:derby:${derbyVersion}" + testCompile(group: 'org.seleniumhq.selenium', name: 'selenium-java', version: seleniumVersion) { + exclude(group: 'com.google.guava', module: 'guava') + } + testCompile(group: 'org.seleniumhq.selenium', name: 'selenium-htmlunit-driver', version: seleniumVersion) { + exclude(group: 'com.google.guava', module: 'guava') + } + testCompile group: 'xml-apis', name: 'xml-apis', version: '1.4.01' + testCompile group: 'org.hamcrest', name: 'hamcrest-core', version: '1.3' + testCompile group: 'org.hamcrest', name: 'hamcrest-library', version: '1.3' + testCompile(group: 'org.apache.curator', name: 'curator-test', version: curatorVersion) { + exclude(group: 'org.apache.zookeeper', module: 'zookeeper') + exclude(group: 'org.jboss.netty', module: 'netty') + exclude(group: 'com.google.guava', module: 'guava') + } +} + +// TODO: sparkr profile, copy-dependencies target? + +// fix scala+java test ordering +sourceSets.test.scala.srcDir 'src/test/java' +sourceSets.test.java.srcDirs = [] + +// generate properties using spark-build-info and add to project resources +String extraResourceDir = "${buildDir}/extra-resources" + +task generateBuildInfo << { + file(extraResourceDir).mkdirs() + exec { + executable 'bash' + workingDir = buildDir + args "${projectDir}/../build/spark-build-info", extraResourceDir, version + } +} +sourceSets { + main { + // register generated resources on the main SourceSet + output.dir(extraResourceDir, builtBy: 'generateBuildInfo') + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 84ee3749cc1f4..25ba6c2353f54 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.storage diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 4b0da68f59b4d..dab5bbcfece70 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.storage diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 20622010ea60a..be5becf145d0f 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.util @@ -2597,6 +2615,7 @@ private[spark] object Utils extends Logging { temp = new File(path.getAbsolutePath + "." + StorageUtils.newNonSecureRandomUUID()) } while (temp.exists()) + temp } /** @@ -2612,6 +2631,7 @@ private[spark] object Utils extends Logging { } temp = new File(parent, name) } while (temp.exists()) + temp } /** diff --git a/examples/build.gradle b/examples/build.gradle new file mode 100644 index 0000000000000..eeeee87812fed --- /dev/null +++ b/examples/build.gradle @@ -0,0 +1,43 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project Examples' + +dependencies { + compile project(subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-streaming_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-mllib_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-hive_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-graphx_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-streaming-flume_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-streaming-kafka-0.8_' + scalaBinaryVersion) + + compile group: 'org.apache.commons', name: 'commons-math3', version: '3.4.1' + compile group: 'com.github.scopt', name: 'scopt_' + scalaBinaryVersion, version: '3.3.0' + compile group: 'com.twitter', name: 'parquet-hadoop-bundle', version: hiveParquetVersion + + runtimeJar group: 'com.github.scopt', name: 'scopt_' + scalaBinaryVersion, version: '3.3.0' +} + +jar.doLast { + copy { + from configurations.runtimeJar + from outputs + exclude 'scala-*' + into "${buildDir}/jars" + } +} diff --git a/external/docker-integration-tests/build.gradle b/external/docker-integration-tests/build.gradle new file mode 100644 index 0000000000000..93ae7e08befab --- /dev/null +++ b/external/docker-integration-tests/build.gradle @@ -0,0 +1,52 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project Docker Integration Tests' + +dependencies { + compile group: 'com.ibm.db2.jcc', name: 'db2jcc4', version: '10.5.0.5' + + testCompile project(subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion) + testCompile project(subprojectBase + 'snappy-spark-sql_' + scalaBinaryVersion) + testCompile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + + testCompile(group: 'com.spotify', name: 'docker-client', version: '3.6.6', classifier: 'shaded') { + exclude(group: 'com.google.guava', module: 'guava') + exclude(group: 'commons-logging', module: 'commons-logging') + exclude(group: 'com.fasterxml.jackson.jaxrs', module: 'jackson-jaxrs-json-provider') + exclude(group: 'com.fasterxml.jackson.datatype', module: 'jackson-datatype-guava') + exclude(group: 'com.fasterxml.jackson.core', module: 'jackson-databind') + exclude(group: 'org.glassfish.jersey.core', module: 'jersey-client') + exclude(group: 'org.glassfish.jersey.connectors', module: 'jersey-apache-connector') + exclude(group: 'org.glassfish.jersey.media', module: 'jersey-media-json-jackson') + } + testCompile group: 'org.apache.httpcomponents', name: 'httpclient', version: httpClientVersion + testCompile group: 'org.apache.httpcomponents', name: 'httpcore', version: httpCoreVersion + testCompile group: 'mysql', name: 'mysql-connector-java', version: '5.1.38' + testCompile group: 'org.postgresql', name: 'postgresql', version: '9.4.1207.jre7' + testCompile group: 'com.oracle', name: 'ojdbc6', version: '11.2.0.1.0' + testCompile group: 'com.sun.jersey', name: 'jersey-server', version: sunJerseyVersion + testCompile group: 'com.sun.jersey', name: 'jersey-core', version: sunJerseyVersion + testCompile group: 'com.sun.jersey', name: 'jersey-servlet', version: sunJerseyVersion + testCompile(group: 'com.sun.jersey', name: 'jersey-json', version: sunJerseyVersion) { + exclude(group: 'stax', module: 'stax-api') + } + testCompile group: 'com.google.guava', name: 'guava', version: '18.0' + + testCompile project(subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion, configuration: 'testOutput') + testCompile project(subprojectBase + 'snappy-spark-sql_' + scalaBinaryVersion, configuration: 'testOutput') +} diff --git a/external/flume-sink/build.gradle b/external/flume-sink/build.gradle new file mode 100644 index 0000000000000..99a0357048896 --- /dev/null +++ b/external/flume-sink/build.gradle @@ -0,0 +1,50 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +plugins { + id 'com.commercehub.gradle.plugin.avro' version '0.8.0' +} + +description = 'Spark Project External Flume Sink' + +dependencies { + compile(group: 'org.apache.flume', name: 'flume-ng-sdk', version: '1.6.0') { + exclude(group: 'io.netty', module: 'netty') + exclude(group: 'org.apache.flume', module: 'flume-ng-auth') + exclude(group: 'com.google.guava', module: 'guava') + exclude(group: 'org.apache.thrift', module: 'libthrift') + exclude(group: 'javax.servlet', module: 'servlet-api') + } + compile(group: 'org.apache.flume', name: 'flume-ng-core', version: '1.6.0') { + exclude(group: 'io.netty', module: 'netty') + exclude(group: 'com.google.guava', module: 'guava') + exclude(group: 'org.apache.thrift', module: 'libthrift') + exclude(group: 'javax.servlet', module: 'servlet-api') + } + + compile(group: 'org.apache.avro', name: 'avro', version: avroVersion) + compile group: 'com.google.guava', name: 'guava', version: guavaVersion + compile group: 'io.netty', name: 'netty', version: nettyVersion +} + +// for compatibility with maven generated code, though default "string" seems +// more efficient requiring no conversions +avro.stringType = "charSequence" + +tasks.withType(JavaCompile) { + options.compilerArgs << '-Xlint:all,-serial,-path,-deprecation,-unchecked' +} diff --git a/external/flume/build.gradle b/external/flume/build.gradle new file mode 100644 index 0000000000000..6e2dc5a974d82 --- /dev/null +++ b/external/flume/build.gradle @@ -0,0 +1,40 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project External Flume' + +dependencies { + compile project(subprojectBase + 'snappy-spark-streaming-flume-sink_' + scalaBinaryVersion) + compile group: 'io.netty', name: 'netty', version: nettyVersion + compile(group: 'org.apache.flume', name: 'flume-ng-core', version: '1.6.0') { + exclude(group: 'io.netty', module: 'netty') + exclude(group: 'org.apache.flume', module: 'flume-ng-auth') + exclude(group: 'com.google.guava', module: 'guava') + exclude(group: 'org.apache.thrift', module: 'libthrift') + exclude(group: 'javax.servlet', module: 'servlet-api') + } + compile(group: 'org.apache.flume', name: 'flume-ng-sdk', version: '1.6.0') { + exclude(group: 'io.netty', module: 'netty') + exclude(group: 'com.google.guava', module: 'guava') + exclude(group: 'org.apache.thrift', module: 'libthrift') + exclude(group: 'javax.servlet', module: 'servlet-api') + } + + compile project(subprojectBase + 'snappy-spark-streaming_' + scalaBinaryVersion) + + testCompile project(path: subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion, configuration: 'testOutput') +} diff --git a/external/kafka-0-10/build.gradle b/external/kafka-0-10/build.gradle new file mode 100644 index 0000000000000..c898dfce495c5 --- /dev/null +++ b/external/kafka-0-10/build.gradle @@ -0,0 +1,34 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Integration for Kafka 0.10' + +dependencies { + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-streaming_' + scalaBinaryVersion) + + compile(group: 'org.apache.kafka', name: 'kafka_' + scalaBinaryVersion, version: '0.10.0.0') { + exclude(group: 'com.sun.jmx', module: 'jmxri') + exclude(group: 'com.sun.jdmk ', module: 'jmxtools') + exclude(group: 'net.sf.jopt-simple', module: 'jopt-simple') + exclude(group: 'org.slf4j', module: 'slf4j-simple') + exclude(group: 'org.apache.zookeeper', module: 'zookeeper') + } + + testCompile project(path: subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion, configuration: 'testOutput') + testCompile group: 'net.sf.jopt-simple', name: 'jopt-simple', version: '3.2' +} diff --git a/external/kafka-0-8/build.gradle b/external/kafka-0-8/build.gradle new file mode 100644 index 0000000000000..f756511358b54 --- /dev/null +++ b/external/kafka-0-8/build.gradle @@ -0,0 +1,33 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Integration for Kafka 0.8' + +dependencies { + compile project(subprojectBase + 'snappy-spark-streaming_' + scalaBinaryVersion) + + compile(group: 'org.apache.kafka', name: 'kafka_' + scalaBinaryVersion, version: '0.8.2.1') { + exclude(group: 'com.sun.jmx', module: 'jmxri') + exclude(group: 'com.sun.jdmk ', module: 'jmxtools') + exclude(group: 'net.sf.jopt-simple', module: 'jopt-simple') + exclude(group: 'org.slf4j', module: 'slf4j-simple') + exclude(group: 'org.apache.zookeeper', module: 'zookeeper') + } + + testCompile project(path: subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion, configuration: 'testOutput') + testCompile group: 'net.sf.jopt-simple', name: 'jopt-simple', version: '3.2' +} diff --git a/external/spark-ganglia-lgpl/build.gradle b/external/spark-ganglia-lgpl/build.gradle new file mode 100644 index 0000000000000..39e0a747ce432 --- /dev/null +++ b/external/spark-ganglia-lgpl/build.gradle @@ -0,0 +1,24 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Ganglia Integration' + +dependencies { + compile project(subprojectBase + 'spark-core_' + scalaBinaryVersion) + + compile group: 'io.dropwizard.metrics', name: 'metrics-ganglia', version: metricsVersion +} diff --git a/gradle.properties b/gradle.properties new file mode 100644 index 0000000000000..53c56bd3da6f3 --- /dev/null +++ b/gradle.properties @@ -0,0 +1,5 @@ +org.gradle.daemon = false +#org.gradle.parallel=true + +# added below options to gradlew* scripts +# org.gradle.jvmargs = -Xmx2g -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m diff --git a/gradle/wrapper/gradle-wrapper.jar b/gradle/wrapper/gradle-wrapper.jar new file mode 100644 index 0000000000000000000000000000000000000000..5ccda13e9cb94678ba179b32452cf3d60dc36353 GIT binary patch literal 53638 zcmafaW0a=B^559DjdyI@wr$%scWm3Xy<^+Pj_sKpY&N+!|K#4>Bz;ajPk*RBjZ;RV75EK*;j-;d{(BB5~-#>pF^k0$_Qx&35mhPeng zP5V`%P1S)(UiPcRczm!G=Nud7!TH`9_!bdReTmO0lO(Zfn zfMqP~+s2VTE#?sl=$9e(CuBOAH2n}=c4idsipAKKQIO=pSsMiScd0TvKs^d1xMCym`IZxg&Xd3ii(^3$M#K)myV2qM z{o5&z?1rtP{gnX6zegV2 z$03xe*g2pGA^BqfBE}XDD-CN?H&?w?kE0wG3~``ie?T+IRmsT;*cpLZ)OnaXBtFzX zMcc97L%tQqGz+E@0i)gy&0g_7PV@3~zaE{g-2zQ|T9d>EL&JiD+E8t?H^#-Hv8!mV z-r%F^bl8v}j19B}`M^zJ>hD+Tg?Fg%no2GYmKkbR`2=|`@o`~1&VbKUJ@ZSojd|ihP|{9&8F<6 zcAvtwl6yo{Js7UQ{l~MjA7Rz8(sYEcRXX(n*(Mp75OBEijo(25zwC)pc=#%f_xV93 z`&d+DI*TAtd3KBd(^964h=X;uks5gf3MU}cR#X_wnk6_crcAVLsTV2SzsGM$h~aq~ z6oX|v?jSq%txd-SHwHy`7x4*jJqH^;0*1`Sztp*aYi4tRMBD|Ryyl%F{IC{(=Y{Y5 zhSQPvqZN~4uvCi*h``E|llqjfc4rnRQPs3@(MnB9jto`dtz!CB-ojaReyN}7BwMp! zU@jEkH2hS%NUB|wE0d;=hS4^M^dTz`r=^`7LNsQkU26nlt4o?Ki13cwDXkQH+)w#uNVQo2o@pEJOAZV3Uf z8WWqpN|lDuGdkokHkKLwmo@qCdV6}M=~DGq+P3}@$$yqQssE{3|BxxM*q?tD3oiW6 z^!W)Iau1CDv+;dTH4Lbb;*)+mGrKg;g)4tHB;h~=3QsCF)I|E{`=jp;ArQuy&zUzA zlz$NoIhz7h@;Sw+#%u~;!w56XV3JkGLOHaVlvs1eSSck_-2#zs%EynXvEnsUsO3{@ z=2B!(Gdra;oKm@A@~#LeoDFC2&V->;dgCP}x`Qm{yZA&ULeNnWvNIGzcgjx2?Rx#m z_I4lu^j~)hR_VQ?`&Yk|{^}Rqf8MFY|1el;E@sY>4t8d;4h}YMj{n$ntcs2Tju6_n zc%t6wvvLifwar=wOlL#;T5V}~s_KU-6cMz7X&7`JeYdHW?WaaBnYH!e82^(58{d#J z&3H)nMCXi0pUcVg^sRt^KZxdFRj|_ZglEw{Ri0EN6_laAxbE8zB=H8KgU;Xtpk5?z zC2?g-xj`9d8MtJf-!H#~s0}tJ>Ksa+7KP;J(%hHwUBewO);ZZ&ry8oYXI%5+YQgNQeyS*ViKL>Yy2`MsK? zB7Y$Zk@YAy#-Kwyo5KSK$lcvER(OV>qrW1VXPo7Ih%dEJZ<|5sEmeC)do0(dJ;7Fc z#v{T#df-92-StcUzRO7OjZ?g-Ik?9eGEDWsUL(f2jUmS9_ajH?wV&{0Aj)-0IP36} z4!4}CW2D{v(ZmPjB$#&;fps(Hvph<>^IORq|0^=eDhYiA%W5!rM_K_y(bsu@*)m3P++I=?)h!HA@uUc{zxJ0ibvxU%Ke8OQ+KDRndS#XDA4T zto-I$zC-%q0v8ZL`!Z;MMK0`Irsn?gZwiTbsJAr&4g~c3FEn8J&tfX(X=3ZOyEhpng#DDSOc6XLr%uGtB2|0=_Au$%heH3&*ID*ZPhs8iJyw{$c z)`ySqPVndS_Rnv2f$xtMcp${1WTLjhW)S3(;l*PK4#cXz09;@vNj*6?$Q>%5jIboV2fgAyb?c(W#K{@rj#6OKF&J#QQ9 zboB4HeJ?hXrHp)H9rx@Rta#*PdvkHJ<<2Asc#ClKA;st5qadT0NZHEA111(&qsaLb zkeTH_h(yr92XkyqqslQgTo(|RN$hhQ*IL7<12W?$+q6R2jtnWadKrIyeA>bj=;9mM zaPK&0{1#djnc2d@4fR{7K872i*IvH0mK#eqf4=iu8F5=2P#fG-GBZ|`J2MyJ(^^*5 z$tQaAS;Y(2k!j10=adaH9^!>^k+iBMVWD9#+F=&Q(yZ5NVJ>A}t>1R@32TZ0JTMas z%42sos08y0NMkb$BsDnQo8nVhd#ksaq8UyBjAO1FHRfW*u`ojc^y3)=(f&PTM`k@F zMoZFS>HCeNX1c@G{<=x`IQr>{11kPK#7AZWy_q&GQYwk*t|uTE9H*TVi|@g8P95wBlwf};`RANPqd z@rf3B=~Q8%Wgl5i2t$W?Ns1WgZ1t0sFVJF73Rc!d@X---3W@e+Dbvpj;l~8r`F9Sc zKd8G>dR2>61(|Bw&XdTlG}}fnu~6{2xsz6Efmc;nRDupK!KI-q=^*h{`b$W=VCBWe|mhK3YN$PO28ZaH@2V;Nbgpwl8Fig6xxkWN7UhWjM%G z<^O`4VX45jgsIIO-R$7F$`uCa6O>(WOZ>i>Gor3X^yySAwSB;0*X`pWy<(jya4!HO zYDvrso1n6V3G<>TnrSv{+unA;cSAWuH!9k`z#^j306@cy?0{jP-p4NUCSRP<_wNHG6^axCR zMECIg`Vz^ja7F}a@eRp)X%2>D5;2HR8kL?^&npLgqU9<#pY)7;V<(`jjbZL5j>8!3 zLF`9aB2GDCO#Q|6P6_x!My0QO@k9M-2f2-|AF33CZ))eL<;OOi;76DiE|3}S)dWs znAG6VC4EKe9MedTp6d0%J409Iu^T12e)n(N?^s^DlM+cUS5h4SEqq4NjK+%c6KPr%EUeiN1v_&WkfIr9Do+Q{2=Ap?{FS|6D&*Au8=PW}~Rt~+3EJpNK(;7R!k z%&9kpQ;0c6#VF?+W0D~mUp&bc96HK5g~~9ch$%)z5Z@Wq`!V zLK|>+`YZuZ?H_46y1zmqL~@TLC0lsiaSZzy-7!W4m2VsY*S@u zW}zeWzr7N1oL)qzKa8J;I?@RxaLmPEowX2BlSD2hnYW}WyvQ$FK%;PTPl4|ftNkVB-&5MvFWP;Yx2`zBS8o_7QfB#MLS!;fRlOU$t z%iCYD|0>%`e)v|$e?d9+U&O=spApZ$`@51x_J1P_|AafS>X$C4YG{A#vQ3BFr-~J& zrNf{=tbRuZqT9ky&r6pgk}1*#cgZ;@Dve26=%&Eu887bdn12Lfj~ET zV5IjzrKV8JQNK?MSC0X660d`nnw55z4~eTJnzix7U0S>nhTZ)+f}uOuozz3gz(ZMj zYs1J$Dy2L`8>*UTBLriGUKTY410qsp)+K??D$!Mr_7=C*Ec`Q^X$b{2>?+1_7Ka7f zd2{rtBr%g2PbF_kHb0yF5ypGWmJ(ftYt&YJ_ynIc9aWyTF~dXbrN!iNKZ_(_u>s(I z2*X|;szamJz{2dl(P2I;%foJLF51)WO50&h!EXP{5|gzy^b(4+nn<;j$NCt?UP_cL zEU1-XN`f*1Gl}Fht4&Pn5xo)Ma$kZyBt2qju1yuSTysFNG^pi%vvU=hqPhQ-Cpl`! zSt+cpY2TVYkvTflcqwieu*7MHVIVSQSivl850|@O!h+yWxUM;@nYhP*y8k$MMX>^Vzs= zqdctvR$e(}|H4~b=2=Dszgy^xqjuY6alG6sJ&J~#*>K6$Dl|Pf&Y~sh$@07*amh*T z_wIHl$BbJK)7B2V0P=;_iRjD@X9TGO4h+mK7laU=qy zmy|9(xNb7=49$9mLHL1jK#9O+t+CJ;3IDtkK4I0H_{C zMZMuxRK)|E4155y$!RGIcAVHwpQK;2>ssTa8^Y0_8A*sk}a3gZD5Il8pD}4GR1;_@6*%Y^za=h*V zu5|)zJI%1zR^G!TWc5ioA9xPwtfcqbul(N(r%J#%9D&&zkuV4xj0VXfymNc> zXa2;S1FBKAgNu0n(1&iyT^XFajIG9aRR$`eVEejDBr(KR%NpK5M4^XoP2r^{5!QII z=({hFfVnU7w+W~i2BF-t(|u6~1LVuM`kfH|v*hfA+X=o<%CrInhQid%%Pn@a#zcB{ z^o7+|r0o0_JFn1}AG;)N{NfS8LI&fnSX*e@KcPEe6OV6U-=oiHjXw;D&;Ui32rr=^ zeI$9fsv_3NJU2zXlqE0HwYjRkF^(*=dmpb#bAGI-iGF7mAYYyfIwG8Z;|+==LK(t5 z=LlsTsJKX@6c-VZ+w{_}$Z-T9i*wIN-d5;hIXRENbV;h#&hIG%+U%XScSeP_M`St_ zW*|AQg18~yjh5x7_;)cRFt3fbcf={_ULTh(DfkEq%mVu$EcpgdN}OOAmZBAc2`mzY-_S)k2M=`*UssiSqyy?xVKo=MbGuFS6XRVx2Djkn-AEcIgSQvVAF zfc`p)g#X!O$~sv5mqQqq{Nb>uh&I-rK1N;7H0mguftm{=rV;MIL=kQZjZ6q_PVrHj zl3gcbRfrb*Cn_KeXJnZ06ZEq<=ClnSMYA~}FVE$cEB}!?;QRYG{63OGvgE!wNV+3U z&{hS7QV6Z6UL=q3cB&(hP~yB{WPCY<2lhJj&?d_e^Y$rRxGZ6c1-iXZdJo;c_d!-U7axq4RNZah&+JQde7@OZf;An_X_9 z;&omnq3Vb#o!)t8oY6U5q9V@2gy>2y91VhMexMd;_^7e(hpL;mFYj^Ms7`BGG-h5q zK%Q)s!X(^S6HkOCW#c?l`3P0RZTN{hKdkGb8X4YR z`Q>!^OJIGZmB9N^htzi$VRW^@G$NKIzjI7BcO^B;MhbLW^3e|uMwSeIP|Zs8x`x5{ zeL>F`oNu5Vl>~_AJi)C>nNM&Fhx!XDe?wEG`x8B+)*W-m%b^U0@g03H+h=q%dPTeRj5TVw;2U@yvqaijNc-yOdE0sdE!oOQm8LWcwb zvPS>{qWI4usc7VCDdgf}W@r3gcXz7#y9ura^7ra0x>qu1l*@W+h%sd>?FNRF3P1|= zZbh`V{x`M!W`~UpsRQ+GS9kSrxHBr?)ej6Lo3uG_H zt<~-{2g_b|`=9T*FAm_G(f&ius6m395oJ6G`(dhHA`zwVV}R^Nn`tA;NVLNfW^Hs9AWg~k&`b-e$0W2lP^ww;)sP<7yihp z>9;T5*j*ExLF8eyk}p*_!`MPt{qUWd(sve|0cQ3d(s?$MugrLsb=F0U=Z>3QNKb~xQiuR|wEjrfU z+ zcuJ(5)AJajIhR#qf<4<5IRe+i9ySGsxMdU+)OPV`G3%LV?Y&rh6sS?3B$pjiq)Tp> z7}ce0mnh`&H|iOQ1Gb0uRJX^%qxxmUZ=$BivYy9aXQU8){*Y>2BZ9l-I9bd_Cb^=f%{ zmiWjT=K4anJYD<1{Uv&xtE!H~-PThqR_6Xk6JGHrBqqGQKH?tN;Y{bu{vvRtU8UsW z(llN(rJar!7lEQU6Og+kl{x(`fYfSOcD-A(iW^ymja51?n_e*XdzPU6$KMxkZcJK` zJda&tRCKm42Sb&*e*F*DU{{L^KV#|D}q2fz@SkOgh!`P8>llzU)uQmjI>w+ojmYH8DUYso9od6~Q z;4`O#*N}<@^x4|bmeg+8V z#FzePXp9j-$0LxzgYFM1c8Sfb>{Ps6=4+{2OSxf{aUT^)eNk{Itv=A6UH-NWAk zbZSac^MG}0zqXgc*4EY)bsLAv>)zhim))%o>em2scaCgRq(g@OFY^hHi1@3ZH?0RyOJwn?nXpt|EK zC7C$;BzYJGJRO=@=P;0~`pP4Xo506(sb4GFpOQ5bebMSZAyb9jZC0r^8@4#6@ zN1(csX24ZP&hIq|Wy)DqPP#K}FbsE(!eN*bF(bfPb223XmHFutE}fA+X)(F@1;%yH zk1$?!wEHfeYJqddk074aDxPkEX(5=S7o}DSWlDe=WSG+rui7V8D&L^N{0Eoj{52^F znZVhj=S5qnyRaUmgH@o~<9q*IvbmW>Rd7#O2emhnur1TYHlW(mYhdexX!Kp=0`-Hf z!SuIx<@XNREb6dlpjM~u!9pOrQ+I)gFWa|Hkzd!e`}FK$)l*+RDtXAr6jhG2g7^(& zS#O*-Cp0BefE>7adHa;nq#sma9#q??1@@V^r}|p+lu`QF`_*sO?>*YP{B-=C+6Oin z!SJy*7wjAkT`h%`ZcC$OE7=@uwGN%GB0FWRvSBmnm)%cUkp!1DR;?)JH7?*wZ@)ch ztfdr-ApH==lQ4l~Yl$%t!PHH1zsCEliydl^cX*{BK4Bwl&; zJx=3~4w4j}SylJRk^&{HD^znI={^oh5?NHd2d1H_jh;qml_=-(7Web$GL)idHHN(A!9a;z33kdFv;y;z&UxF)%AeR4wcX*@Dd`TbAx)+%j<6 z7<;WW)i;(q6bO+@sG3~TKii83s*~Hn6H8`K%#A+`Q1P~mlINmHgw45@lEne3jJ17P zZh;hz8;0&UQZnl9P!Lf~yjjLORIsk_d{Ii_YQEQ%aR@^GyDuwHeJ}ihLhY_)AdGHg zhFSd5RVy#l^BEgfnr@AE7^Ft-Psf|qCcFrebXbLZme1nEm7HrU@Z!uVjNSN%-b3af z7qZUkhP=v4wwvDaxCFp{JDGra@o7D-oGogt?x{pR{FrAwaZ3wj-{uzAmKMqt2&XDh ze1?H=?{VpKPF0hrr{{u7|~@}3h?pBn#AQcs3}tn(ld_+ zqfibU?{%p}DH=QlEiIzWnzdn6rJbfL=v^pfoHN&UQod6xHxUy033s>27|xW2!iAIt zRMTOs@W ztWJX=uuAhKLfXd7Xin6FH>PB>VwRXl4F(L&>kRYIq}#{TDkrvZbm;tKBkE*r5OWJg ze$J|ti%j?`N}(DXhW1uC21DuyX5QdoV|XVRKtk$BO(Z z!r{tOk9B6*;^R}Wbbq0T8n^5h^;eF6;UW=V@uJ%kc4}Rrjhc5Tf!euvA3@BRs^etQ zTvbc9z2dp|@0!7GwZ0$+)pv^B;=9vdN9L&x2Rdrsyn9jC@59nFpfyrwenQ6#59tMB zbxBZXtF3No6)I@oO_M(F>|pLUp*fD=$lgK+TWWnF{Y4KN@A9dy_j=>>H)bJ&9@U?0 z-(8chcmAoj#}1M%`IE2FD2NrGL|<^YKj#l)D>KZ*V{He`yRLk7DCyeO=TFkfPHhf4 z5|@pgWJH9l&&_3*O#Cv^2K;f;@ziVC$~X*WqHORbe`PI1`*AOp(@oD_R*Ppc4F~vr z9G|JbXZs4my(CQT9e&C3$OA;PJWyy4wV#ru3Lb_&?NV=uGtP(tB#tEv(Qle)=vYk4 z_2_mBJ16D~hlCE6WQ@rh*$0;0&BE>^ZBhShUv^)=m zh*`f{oNA2|<*ho;N|huZ(U*r;ql>cY94QNLs#2$sN*iX*R7#Fg49~f#84)>us9SYM z8@F_Gk7c(OJeB~Djj!~LB-D6=PR&OfBRuiHS{2D`rca#zAoHu@6%XY^snOZ`Pe{_!OQ!llj z(vX$_-^MCRyV?lYDAQT~lA!9eCC;-*J^9Q>{{U0rLXqySl({*)}PenaIaXeRr|kqvv>R|%fO}InxZ~VOfnRIr--(j~C;1hh zD*m7*eDdTK57}_}{UZVr*caO06;H%}gb&))zDo;^G(P1=7mFuZQv=`$A05DWSR}Yj zz8ZepdD~0R;-d)k(Aaw^lnN|J!2D4^-n)OM?w|7@Y0vZ25q!@+PfU=QX}7$BJdUxj zdp&-GJ8nv;ER~ac$O(wj00ZDTP2$N1C|hle^8}xhQXJzom#1|R)U+mBjk=y|-iY!q z#0nokHC_*isP`5cvj%!?Ooi>_cRXO*x2mYm_L4oGpC8HQUTzGFv%d@~!e9%SuPlpj z{+Ksf!<#*U0k`m-)`OUxR@_@EbDXB26M2=)J>8l$jiL)8FEthy`J}zQnE^>+4bdYbk){#$Ae;MY{p}>NLh`~ z%kMR=#_oMtE+mw(R^lc%8j^q`_5VQoF zdsg1*Je;fRn1)0{M!eq2XZ@bD?`m0tNP`8$^5dk~b~$&7lVu};YMm&UAe?26^F0+) zQ#M%s*_)O?gbLFsB}@In8QSnvrhwl_hj}eQ#Dg$d!fkT2Bpahm|HITp`fg&o5lq3; zIhE8y*U&7&SRM}j)U^6QBW7u5?zLzorLgcxF7fF%IbQblWkITm@JWRX`9%5#5AFm3 z?qVtRQf9en5;}Gc7cg0+4t*tiyZN8fM3#oZS>Ty_0y23t91&jU(~bv4MIo%-OCEV7 zdAXNo%|)mFOFE?n)-|vp2eT+$NF-%FZ)ZJTBUbSk05b~^3%&W1_W4{Qt~t9odd5QM z`Gc4TcYEVcVve}3n#zgRpAFF!s0U_MxOA0)q)trM{)o)oLggxismMzF;OEj^Q5kik z8U%h>Fvp&;ufkk5TM9Wz*OYP7p7KApanC7QeD2)v0iO3c8vKIb$02BzG$mvFyE6cnmvjQ_D+w=-K2%xW?Oqod6 zB7Pl(^a*8csuMhcKhn@tK~z-am+?uA#K$Hy!*QpDIFp;!7GbNxPhhS)*SU3=9qWo2 zlrRVt`EqCX$@&P93(B0audju)9=r1FK+l*9&iT1wn$MJGx^nFLKC#uj7$2`SLY%5e z5ZVhH0FiQ_$qNYobPI9S))@hsvAm~!l+P2Ld$W0r?(zDtGFM~l&o`%k#PGW1`c9uT z>??a$mVcqFF?ro?X7rTop_TB;- z-W57af`0;N3i0G!+UI{3p4KevSam`(@cxay(;!sWX!1I>p<3VQ>!lK81nJ%b+ zaC{l?Bb+#xHQjN8jlax_iYvts#}cHKzm8Gms}{A*qiP!gaY6SXmN-qK^hnk2{Y7k zpFB~8I{Ao7f!d|!hCoNV8t=r5DH_O%vQ-`O3Ij(>ItU^Rdpg~c^&*XyHm`-oj#j$< zs@56Irzq;y_xL50pG(TO$lB~^%`SH3y71v^^PHT<)Fe!in!K<@`4L#QKo1UWCIb=t zsSrf|{N{PF6+M&c%}mwiq0W$O579{QuZikvDg70zm`~43QGO-r5Pk{^lf0W@cj{ei)J8*ALTs_Cpj#}WC zi~dg`S)L3Gmwe@?Qcu2X?ANb%K6(0%i&Xuf^x$WIkRrJ)2)z*Dm7R3N4yu5;J@DJ% z7QO8HEGp|}R620#Xlf#k_WQ$EfvsI1ws3xNbliM)pTUuFnkB3bYRZLSJ}s3GFL*ww zxCqqp2--kYmf9t(woiP*gSs97*`>%XM1f~pqciQDv}yl|R>f0=Y3Ec#{H^n0b*cUf zT?z`b6|cYwVav`F|FlPc8)QB8n0b#WsS^%EKR8`vz8C8L6UjM}H(wCJX}}#Cy?6)l zat}6oCqnMj-)GcV)Q|4apKY~;bkGOsbd9*rK}hyiVwg7);0g8OkJUmj^nu#&Wb_FS z5;q1k>;crXcO>Jx1@HJmgwwCUFnFk!9`{KFcT?=lYbe}3PuIMZW7h5sWj$Zr^BlSB z%4zYVvbVE{aarM@2NJO7Gw;HP>m9}(Upz3!lm*cu2Wc%DVucXK>X1q?uj8(HJQ3!V zH5Zbg+|b!G7#Cs98{?Lz)=b?9ouh9*RPDS7)fF4rXSx;gGFo9;pdPe&uUvoVIw}%D(mbE03&j@<0#ahS#Y}6C zzg;$sg8j!*_paDJnNWjA0=H@ZrE9_p`T0i5lkMTLb2za(w`2TOvGGEQGJny0X}+=G zIc^9V2J;y%k<5M`x8M=}etGa@Bj}g|W?6U1l0!fE*BF9O zF}^F8q0pPw3rnD`Cu$zYPicTy1ByF1g1U6(!G#XLV)}q zAn5=>`!7;UzfkT=LZNd~RiuDfibAh->%HQ=!8V?TnQw$@@{!w_O(zDwgk8>;;Hm+8 zl#60Sp%qZH%iosK?uTL>B>OK|n=X$T9`1z&gVyfS11B_L)WxTSP-; z**~VIOErBGO|wcRK?Z##|U}R@;6~V7UR}1!CHv0ERDFhD zS$yjwLc7+Zqang$sdZ(wtE9>m%h$g6=5H?YkXxGFY|&C@Bekzio2iik-la8wIYYD? zd07T!tEv{~#%ZUUkyPkHnTeUT;?8!xt5DA(X<25(AnGPQoq;pz{QFozFr->fl}l7KL=wwU0c$$0u6|Tn_0ut2k%t6$hP)9U8FA)$YI>KADk7I00%L1I zYcZ~iyUCV8VSB-!$*VTizC66F^K@f~Y^$AZX_?R=SDnRP}HCj$}I|n5d4*6S5u5RX2efSh`aD zC&D6%JG$k1@||c5Ek6vkOqA2j_uxgC>~=dUOb!t z)k|iO1Ez1pL-pzrsG4hNmHDd15X9VKIwubXrwo6&@`@dHvzzuz84AM>0`s;8f~^=J zZ-p?3-Hb z(WDr+^=6inA%7u{Z#8R-&)(D^n&^RbVqjksyK+cLGE`Td?i{WWKzBukjHI{d0YW$L zyP0A~m8zHpd!;NwkYLNY^aUAtn~9FOib#_*o((EbtnS?Eao=%w;f3qjtYqG!g$ZI3`1M80Xw1433 zYDuT%h~5S7xgT!7BmQdwUvjmbZ0T*t0K%+9$P9QPGmyEzXEd?WB=Pso;#S*e+%jtxnvqYOP;|Zfp=h!2kf%MR+7= zqGGk}Lgx^XfkhZ2xVInt_k*Agcds+D?9E$t@BvrZuf4Cmw{L!9J|Dka5Cvcidr=;a z==`^l2XsOJKXd)J(M7QlAPV>GwK~V*+rb^{2^|m*@jWe&&^s6+rSYDQ^n_H848ghf z=!SeuImw26j-NhEJ^LGci2@NPUqzZ8j57Vm05goac}pbcX)}pAEqx_0{oc6 z8=P1J$q!$?Kn76z(ZIe`fees$sX%?yQws0*twdd+*Ow1p;cyCShuLpwlJ9MVd(cIg zd+6kQu!kRSK^ZSYME;?KkoTyctbzGd6?==g_}DksIQZxIc{ObIb|s5xX-)2y6ESs$W_f{ivzGA^!rv!~r{nB%Cs92! zXKk}PxU;-A98m+b+z6F5D3dnrRIuE_ACKCfjd-^`g|YBr=)6-s`VJpTJ3e+`EkKEw^^iEn=}HPap7lN6VM~4Fn$X zutC%X2g+Hcm1c95+mf}-GN=fN`3h;OEzTc`$%qs~2*Wfado zPzd<{tWY>h!$b`Jnc73EI=}cSth`#km4MnUlg+@_MUIxdOhZSKN!zt%hJ|sxRQADk zFUbNBn_vbW%%oMMYqxL88&xk7gJTD!|rdWQ_?n zcG*h4WcftfU7B*^ zyY;4$6>n_*L)?0fo}9@|C@S7{4d^Xh>|vOp5wG(Zj!RjtDxCHSve@dHdvl9c7BjTt zIC$wdXJZa(ys>*f2KSn+F}3)p-H*(uM;l~DN0mvWTB$P3 z9XjJ@R28*M3U$A7Ej1-T2Hh0_%c#3^9?yo^x6^+>DLZe~#MnLu0&OP8>q;|((mR>R zJ5cfFwZus71_JM?CeQoIW7#+QSWIP;)}&MK$|`$1RpSAHhAU^P*K_W9S6MzDu-kVT zwS%Q3g^RFVH76n$0&tc#C0D&$Z+A zV)?!i&p;#EX1K%24(pd_RiUjSdzbcw@h>av$Iixg@X}!=hrvzbITP@5uTpmfYx5M$ zw!ElwFZ1IsSS{^@hZh-XZWfVM-KSj3Q?*2R5HuMal(y6{efA5~ z1T&LnfJ0_jz-Suospc$;Ad73` zU*~dB@=2(oS@R*|EEY{oBTPB27Hv@+i<{SaOLxumq#~@R6R&mBJr8g+X={q1hNdUC zK#4cHD0yiVNLDS5+pk?E@ zs=i@E$z?;yV`F6OE{##0JED?1BA)FO+uXLQWz)2lg;;1*JQq%SIIuwCxcp(h@UZ^n zY65>V$ftP<()hOq-99k>a)lGWl`}+10G0B9wk{_GYL>hexp!TZlYMB|s{{B{MVzu! zH_*q4xaq77m^Wm8JNmc-E?tE{NEk(hJ*AnwB8o2=j=IqT#DVJ67uz|dV$@6i1qWKw zhEU1Y2jtoPgf}pYcC&NwC*0&=OJ-V_VYscGPURO#MT;vN{XJOALTgdXJ6|k~Wti0D zm_ycwy8NrV4?NPkNa3@VEoX443umrYw?Br(^x-tNwwt;U0=&|(d)ZlJt+JbF2X${0 zGnd#L^TzQnc*+lx=$3y}1#1QCo}v;{Hbov+$L zIy3nf)T*-#h8F%Tu{n9=)>ZZ12n>H{ z(3K&i=jM(hT8Ctl@Z;WeeO)<+kv*udON*kb5&nl;GdIaQ@E2TNIO#j zqsR@_F$d5C4A3F=bZla?E@R9+-RvJahY`JR_;wm_TQV@7nv_1ub8HLY{L)Q zX?Jd&ASuon4BbfjVo&Hk?tG-m#&h~1YY(;w=6jNgzFa_L|BbP?ii^9;wzUba!QCOa zyF+ky*Wm8%5TtN-cXxMpcc*Z73l<0;Z+Gu=y5GIe_uChBR~IE~t^b%~jrq(!ZybBv zwQF=k-wQqN&xDpW!dg-ze)0}Ndum9c=kfnMg6BR7Wx#%)c>UctYj zI2usyXd~!5>?j;+R ztcEQ})Fp88$?c(Ka#E~?%U+;S$hmo5Pn~x0yqLMUa0BUtesm*FU3PlBcX?bR$n)Rd z6o4#8L{T^I(xcH;(AOy3NTEZZ==iHsaqf>)yt_wjcBu`W+qV!tJ>Pml*eNKI z*NxwZdAQevbL>vRZsq6Cf9X*j`r0Xb=iQ;RMV#VR-immwbYlx~eY@`q<5?4a=$;es zRG|@!SgHYNfF)2+ByHOaL8N|;B}6PQ9STt`WFqb%KfH_8A$T+zkBKS0*+*{kQ|aSE zCmb@OCKwq}@y5$hxRvD1+fjz)uQFuGR=LIYUXGcOOrQCE3jy&XG3Q5oi2T?9gS6PE zSVkW^sqpod?OQ3La~nIv_1&cR>p2~1QSrvzR=m*_=%xtkso8^i&eQQN+#7ig3(wgz zgY~V>N9;i=UH8as>Z;hc_p=-MInd$R&hz!@;{5z#jRxt2yEtcdfQgSE<*Er~?s*jB zXFNMgcH=`UTkePw;5%hZXIDW@Q$s}o-#|&f=-T%7+FZ&{`V5FEQtC~dDDf&AP`L9m zC~$9^BgUd67t#IUt;JPj(zz1CFl;4Bmi!UO&x_(K-P{?RC~}Bp*R-&>8wnx8Xzbkt z7$5IVi6FQ24MDS&D)D{RagO01KgnD|?qr*b&cl%JDsDJ26m+LMeg%Cm3q)oc;K^qS zEGgQDNeEF}l#mMbk7tyv;FH?yci@X-eXiryo{^x%v1J(biDGOz?kqhSM%{&MD zndH(?z~n^%R>5tE7o~oXyK+427*`?N9C_Ey%POchfv>+=+y4&5?Ik}#_pTylu=i{v zqy3U4iH)h+K9A0#|1F+f?65I1v%2HdqAsWG-Q+euLw=j1&}Gw<{nbQ#C)@@tgk>w_iX~Pb z9ggj=EKc+;xmp37?l**~E?XzxBBq@n zX%vahMtu8cBAEVr71W(-rG;b!E_RNdbW0Sv5UqZ>NeR@H3uT7cc)oFm0y4iWTBwjL zWn}4%GMi%1hK5+RK`Sa7=+N3XA@remIit2z`6zXfFmfr;G?H<)Itg$z#k?@7It^rL zRzeg1}G&58g5hLuSGb5qO?ncqDnuu71PVYObpw4o7wpYII3oA1>GW< zkcHw>vt^n91MdgI}HISUKD#lo8X7i21k*=5)OK@xmFYr4YvPm1^qRo|t zx#WkfD8|}opr-7wi&myQ?PGB7b`FUs*~MNGW3b3Sj@X6X<2=Z4M5VXCzOZfyEi64b z&fMyM#o$|TagMHE+-GJWSW2i0i(5=ZZlN#K0@P%}M2m?l!RS@8d6;l)W3Ao6UX&Si zFS|!WXd1iD8PVV{RbEtU@B!=Ypdf~_jKHhY#S2FF`o~*~_{|tXN)wl;EA(>dF;j-C zyKg83O*}B6oOn8vt6aV0keW}c24fH>JQg}K4&`YZE~idCU+QiML@^J)QU&R9P#D#y zF#*5w%A2YBYWo|ms?8FIbk+|m2Q5VL&UCg1(;e^KckoO=AP~?xbzIYvE698mMW396 zlLx$bn3y$Y@7Opq=TSDK&@I+Jo;eyXq71vsq!dVzj0H})+`|^EFskziD5d{k2eV1I z9IT65*0e^dgu5)hPuJx0fl7$fC9%xA2*QeXOK)D{_9F{9SqA4o`w9F|sZ!aA8%&;S;<%gqRHjwi^5O`VaVmhQ${Rk#EYrMQY_K^ z$?dT~5p2_3S8Al!U=54VTFhN%3^oz&%cr6}#?fF3OG1T~HCR%Z75KS?+UgF=cF;r~ z)x?r~E#`lY!|h3DQ&lu@rWg!KD^5y3#C;*e^0;85Zp7EsfDD;eQVY{!$YLuFrx#Gg z8Ax-BrDby+#X~@{rO4wQbH~Lc6@#%$Eb>Aod`Co>T%aMr^|TK;7}`WQ!UW*#^5yHY zjh80o(vda?g*9n%pu`W(nPeLv7l+_JiN|kNl*UtQjg#vA!)=ONG>%zMbHmKh5I$Bt z_u|G*(Zk(vSF!pCH@bHOAEAa>Gp@!N(Rp~L+1SKCEX#u>g}(q`i_+9 z1kda0UH~h5v4aIpN>QD~j1T2$djC=0FpQ2vG_w|{_WY_RVmTS3ebF_tHLlj`EKk@a zs{1mRdfGCKj>~egs@%5Dm1TVZv%g+HB=n^)*=rhipiF*+haaM8r}|hdF#I^SJ5%yr zhd0K)$D6LBoqq}mgP18t6NRR@G2?O z^5vnT`obE9M@V}>x0BaJLs#bs-|?sqS@Ue){Fu!nLv7aB_S7}-vBqcy@)h-sI{%1_ z_KT`aVUo-cPO%fI4{mWvjs9W_qm3Jj!4soqQ4~Vrtc0{($z{D%(;kF7@h%OF4|0MA zQbQ_%dmVnkw=4V0Ewd8(_f;fHDWh#*sdIc%#=8YxEPsy3wah`*MI$9~aFNVM8oY+V z58N?ee-{1I6#dgY)kB;G44F)>07s1NZ^+iBk==7Yi35$FJzf7V zIMhF*=l?}r%Tl*;!(BoLuHa~f^Gf1<`JxyC&PYVFDg_VoO-=}v$6t_%SZ;4vR(p7! zD263lQJj<_4>7m1pL7bH>~;r@K{pX^^}#0*UZVccJSZb%W1~@CEky_9hCkeNP29gY8&>G4bZb>-K0PgFCJ8gq8z{Hv{Z% z&;l5@_GmGouX9+w0V9U)I(w0OVU*dCdsL^{NVkoXe&k|}Gh#P7;1s0pdU-zMwLCT) zR;vYSyK)ok=xWB~kyNBMZ5G&1RE=S>)6}nE$tCuhHV2wUWZRBeVA*^eV@&T4KYao- z8>J@etM7A@7(4*0RL&+k3^~(YV$bu9)@#Rx?= ze3jW(AYMZ^vk}*kf`#Mc!y~SdQi!|QxZ~^%Ew*}6dZp4A*w@^B#sC|%2y(WS&wldh zD4iBDGg=j|`ii=4Cck<{^>+oi@J&M(Y8?VB=HITKvm9-(E2#I913ZSFKVKY2Se=Hx zJe3-Ch{*{?p`=}7FdrpaBru39>m8F11i)gUN8&E9PYLndHdk*H#r-vY296s+4#_nZCzlQW4W3f}v`#qflyLh3FkP(s)bMd@6~${pwe2gSWmhZr**B{T z5En}1-PoKH;=5FCS$31_wiaX0pv?A7=y5rlOxGN88=X>E^ zQwDj#0*=q0i%R{1f3Rp(89KMi@xGG>xVdzOD06R(Kq0@Wja=V&;S=s5a{mckK=v-& zmw$EwCp=F@7BIOuHel3WFJ_RFz$IV&6}ID?EB&ThJKvZ1~x6MmPge5 zt(SYdXt(59X5{!;Xk>fv(4V&5h>9RVHi}I}qrn&%#?f~i&jPKw+a51q%CmSk1MDEX zKe-r6bb+qh+%7o2ekA2s)~|-R7IaU9lYp6gN^E_a&~$HO*Y`}0or?{lth{q)u=E;| zI`}+cO0H^nzhsK~>!OtU&k2N0OLVYInlPthjkzZ2?zWYiku)sQ47LbfGme|WDV;l* zb>mOd&2kDRZSsW(mrUaRVkLmJONrT*|Hs&Hz=dkdo6?%J&MdaFQHRVK`qY_&J%MA* z_bu1cSSTyp`fE@g{Xd~mM@Q{e2?*+Q0g zVIH8a>a3_Qfq!l3jX8{A+0iQ!cZm2NOV?bksI0G3Rp0dotW5Q7al~fl7a-JGHl9n% z;oE>%4dION#<$t49%%DllWnO8T5z?RpbziccFCUVExL|4hwyOGgT(ixrM1zBV+8!d z^X4{oDIEy<8`?G=1FBkjj!DT+Iz@+~qsPvnA3m$YYa44D5f~z?Td7(CU`g#BkaT5N z*g;A#HjrA>$k~FNN#(p?3L%cbt2&LNuOsx5&k0 zi`vIXe2K4EbSAVsA**>T=5S)3*c1CSl;av6fEaX3VUx=A;q+f7gC%w8uw>&q111fv zZ_bP?e{e|Fsm${(@fT7x=0DJ#{)DtoXz-Enn>m;)G5VqlC>JHg<*H(Yt{#e#jGs(-y04=5aS>Vo$1;BY%!$&ez*ZtGli<{b0rA5}r9)af6 z>H94&tGZUGoI89eu}dOeSbH=dzRfAM3u-hI)p$zs?nqeek+n_bh1t+8m z)Hb4(LrOb-pe5*2>HcVG!fJtDU#w>qFKY3{eoZdw1yHgW!0E(yWylc8FqYQA1TEXD zbf~B#yG!7!%30~Ut7m9o(CKG@2tdj|R&9;>!RGlD3(LDB$M;FGK#^s$JHht}O6NdN z5j`8AY_jR?2}2T60dlmdKv23$I4Du{j`3BsJ8Ex32ISc(jt3{Hr{-@x%a%l+Y?3b^ zAZlPBAe{fRv-2N4%im<|j0~(rEIuP47KSeW7qFA0q%Dp6Ne7%Z1ui*PI5ah{Yd3m? zTAi=&X-f;DfD<Q_M{%F8}X;?4MUJ}8nu+pvwssN^tipM>X-sA+cApuG5Qmea`I+!?Wz zULdjKq+Jc-)2)X^`dQ3fOXK??zua!=6M0RESPpv-s+8Zw1a+M~+)B=kS>MO*EY~-G zqh(d07C`WlE$e}SMClh>Mj6YS9Y5I5w~9o;u&J4TY_k07lvEV7LnK>NO%_n(VeD4k zG){-&tGycc@kilbQR7!qRoa)@uMNncI^CHCej*vD~KpBG?G4FFP`b=h3c<5 zL3cO5n2TX8Ns;w10gjIPBve?{hVeUA_R9O5-CF_W@8-PrpMl)(WyZAYU+qv3Uowv` zc<@@i8N{_o6&`Tq9z27D9!$ZXBG12K(1xTbPA-}(BGM8IhS8Xq8qs3B1V%+B8<7^+ zHWC<^RHEO@de4X)Q)d1leX>^6?%(m$myvMyrdaKQ;co-G*a_oPU!!}z9L&OeLum*7 zPmeF&ND3Dl8U#cd5d?(ze_jdyNx=+O5A#Da!}c|8=wRW2=fXhIpo}d>u)ZRJjc;81 z7VEmf+D8kctKd-2vqlS?v^Fz^QL*VN|7F`mpjxPht{s!yY)v`4r>erMRi&o=hWUoM zc$@*eKvtV%c!_@cy|wR^bL@3Ik>r=-2hr>LrfgyqTnolEx5{gqRP6DN}YZqiH^9hZ1}POOQ87awyCk^DBpT` z3l3r4lFjkr7P(P=KPd$5rY|V&t1lX0jgKn$b(q*dDD$8`!MP{PJ zE5l~1+~7lOtKRTIY^&dZLCB9i=Y=mk$3>Qy@RIJ2pb!a0q!k;)n7WKG;lbeFqDEWM zUbIwY8gyU}?4CS_?RN6mrP`90btKc}IeaW0;4A;$yCp4flat@16AYKs;ImANuE@Ch zgjeDskVwH)h!u|vKg!~VjeNZ*;rLlKir#Nb$sHWzp0HE zKRf85#yP2R6x6LR0UblQ6mtru36wz`)a5ZMZllv_vKg+$cXf8K8lG(R`W7itv%jz9 zv?A4R`t4b~3Ju5+WhP@p(!*%A3iYs7i5Z);*uc`Bg?t}Bdp&KIIwJcE@45+Nsj^<}B6FhfzPXOMZwm$|zrP`#no437g$~jT=c@NXr4r3@7zb;@7=b}XQ@+T0An$s3 zy-I}GgwuAIcFneODJl1gC!5vLa4zl^6=rI})~MKOwK&^^*Mp~{|Avzd_dePHu_OxR zWb)10*H+H|oQJnRcJi%1k+|P(d`(SDgv0VMSW)SSUaRS@I;h!^H^GA=#VyTK_(!gb zfYYG+%8EPLgbXQFz~R>)fb%Mo_f*X)0&>nVZ#B25(?d$K$|%>Rf;cJv@dw2r0QP_k zOmz*loRlwq%RV0tfGOWV zh1Ja{Gc3yusN*N6APmm1UxMQcFeAHY`&woD=fh3C)mv^D8)0T20-M2-Ga>*$mbt(f z1%u6C!5IU0aO@~2&ig$|g^hi%FkNU#y1!3!u9J2j9UkFJap6$er-4nKm)0;T+aZzzSI<)c~I+d3DgJ zBgQ(WOG%eh-479Awz40?Ic#qbcB9V~pVpl$g5StF&L3iwtG-RYO7M2I^rT*EYB@}5 zlr_g!Sw!cg0-U<5zV5mO>=Wnpf~@J-rwvuM1B40%d&fu0Q`&tx1iO7|^cpZmgR%yk z3R=5%0-k1ulIeTxr1o<2aCk%Rd4Trb91M(hfeW=!m(`+T%a$6+4Xmp63wOl+%y~JpiZ=VoB60)9h8{km2x5%$kR_#)es^mlcUlVs=*0cHly>6$+gT)P zh0gTZSy#^7GE9M#%m)`!ff}C}tp13BHR<3_=&N zhGv3Ic(b|b)`N`6#z8|8f5>iIN_kVdk@>3U3O0t2@!QapkmnpkH*md(+p zeb;W_Ru~AoJ_jTh!CuE~h&ESGm1Q@S!L83OCoMmdp1|3!FIQ_3KGq|gPHihQzCKSV zg;&`PH!e@vqQY+5n$J|QnYWd|K3A{+YYvAe)1X?2DDx2HFN^FgdwCser(jP74#!{E&2X00Yr`cdQbzgR(vimiMd zNA_!FBf*@-xW%lsWt^?AddX)l4RS4u`z}JI?_+2-`t*%_hFGZ=Kr#Eu%MP%k)L($G zmJ4d3&3jkMSjX|y*e+@=vvSxMI=NLgzw|}!zLAC4^qxh<=~(Ozou9>_&b@!|%YIaA z!KkG}X@w_4p(I*&!v^|5IA;yZ25P;#XjY0hN}IVag2>$kgZ$Bbcr_#;B*Go^li_D( z@pm&ocQqt+jYGXAm+EL^9-TvKv&W6w9thfs2KUHIKz)is_}cbNwDt_G{6cBKDQU_p zYRfCknnMP{ps>$tQlsP=T)#r;6N26xZw zKx7F=It;ntA6NaNhJ93Fl3J6j|F}!XFB~2F72h3((74h!`%-q2WUMX!@V^_rJG`Nz?Eck!gZGL3RRbAlc8PT{kyZ{bF%57iM2edLIrX5&C?+2PYI%I zXt99XAiTJJm?DmB^RJI}#5Q}!H_^xAW!E=&)3di-GJ>FVyjz|GE^vr#Nh25g#3JFv z;XjkU;_aN_Dfh|qxh!lASsQ>JbwyBB+W|-gsv>tNKR_GVL`ua{xA-x#D^v6cF*-SA zgy^IqVrH*F($Qbn1$&o1{-!ImiF^qZKJ|p$=j$(U{XdcTbZ-A-HPZcqRB!)@#i6(V zKP-3U=_z@tDM?icN!f|X3EHvAnSG_lsTpN^X;~%4S_Q`1MrOuZW_AU8R|G{yW=7V1 zYRd5$DOq}|W?2R)nK8*F+3B$ZYU(MPu}RscL(qSt34nd_zT?-~nPxse2;Qgc{k+cv zF?6x8Hl}klF*32RcmB6e!51TFH$Vg*eEpVG2E70o50kTr=lIgy6AqZLFN$?e<0)rt9fNOYGuj}+R!Ob5B z>a#gOe&>t)>d!@gDVMA8>96f(EjrrupF#rtS)(#1^q9F`VLKt~oQ}h(Ntw*GrQ*F% zNmvfD^=MC8BKh2eqbIo0+itJM@4)_GFL%kHP=t&qL%JK4{a!*JA}6P=Na=+yHxCHF zt^HMZU`>NPwoAy1&-40ky8o{q;q!a{pIHl^g6|gNK(z{uL-Kc)}-_4e4&hh*MQ z+LrPsZHwu@zg^JquZXyjv#_0w%_o;o#=_R*6T<2GN#_5ruy~S+?k9#HjSmbif~Gmt zrfPc@@vy*ogFj730(#C|s2q8IaKr?A#YR5`Ubw)oe>1R7IIWHU2UEG)%1~beL^mhq*kFr#ttMPg&%1fPIIwM)17ddY(dM{ zm|$7$c*Jfo;UWY`M3CXl*Efd-R7=iV(J%4Qjz&GvG3^8xSn2gn_H9Ekp>>@kCSg}n z1)p>MY^Jm@X4Lbk^Z3!;%`sayVqbB}olO=JP<^3q!Ja=3Qe-KJ z<-$Q;E<*?*5BcqYTTs~5X=TR$b>6UVlCe{Vv+>0vo~VIESk&zgxe7I2)8d1b<}1y& z??NZr$i&i9d2Es$IGSSYYONr@1?AA(%%h<6acCOJN4{u?hF<`sCDLgNrmGaQ^M8w} zk#@Ym#l>KTW}``wng=U61g&mC-(D}k(ijVRz*H{yl9sAq9m`Qu*=23j_IAk22jwff z`-db}Ovk}+vQr#mjR>R<3lu+799RYM4mEuD1Z3Bx45{h2{5xgqucgTR!z8Boa|xb& zzWx@{{cGj^+Y(eTaQx)h|CcWt94YwMGz`%X^Cm4g*TeQuE8!UhG;q;A)Irx$Itx(A zU-tC8{vjyJI_>uS_X32bTj%fYprc43(83}?C)T#jJ2q9RwWG2dQo2`GQxq2X8!n{R z@t^gm5VrdHSub!Yk=Qh-s4l6rk`_HeXld`*BMJF?sGC;9RFZQvKJjUskfd(de$j3| zY2k;2JDsw?cv(JVNXOn*=F_LuU7xd9aK4#e4tImyciMp9F6T66vLjC>80~na( zSo`z?)DbJw*nrCO7~2e;x5K#aM0yhWq__QuJ0Z;3C3H@>(!;e}M3E zgV52L4pavfLlaoZmMW-GQj^U@sis7jaEI6+ht^#Xq(zuU7#~&>a_l$eE)h~XxC0-* zLj(0#+V)Sr4(P1aR}7U;(G=@#GDYpBt5!HDQsD|cy^}|OPo^3VCx(B*7!YbE{BD~- zTyLsMnImY?+O58^d|BeXQJAn>-!w2KYm2Ld_?7euE^r>-!mplJ zT%MXtb>FVVW!`$lPh7G7Fy#%MAzeo$0=HV$cA||Az}Qm3+(XKM9iMD2XzlXd+5d6n zJ&0_X&H;;hj(K}YuFVZ`0f)#Wj69(uU#Q8@Oq5)or#TL_8c_TdK4ZYQo3WW5gdX+fWum-4aLQo-t7dCMT!enzWjL%H7HNP zV3~Z}`VNqJa%GeuFyizrFakKW3YT--=qjfenAY<#M4mMn))d2~5Cf;MEnfYHh}A7L z8h zh^<&eLd+I{lpnykh;*~L!yYL4S?^IILpVG5|M0g3-6@kH%*Em-Y)v#}nB>gs0ggx-FUGem-(sOAidI2#zY zh&d;jIr0lCjRE)_rdq$~Jj;U!_2a)q^8ecHot3i8e^H+UKOa1N|C=4~&!w;YX;0aj zeeMr?gHH_ff6ao)QknUrK%@OJZ8tHn$-67iZwk=LQ!u6?`iz(qhQZV!qr!)66<-#^ zYr2kIB|q2nj%7f3-GYXZ4>$NF-cGqz30YZkI&Wn>b$eWI*lj(G%rI<&pxsXPA{`-M zS!^k>IyfreOR5+j~VywzG;xQb0w z%vugs{rpJTD!(a9AwuTOB(Af-=UpDbhvy*Y&(y(tt2UUPr6OTg*NX)iA>l947mYoQfXk^i!S#*U@1J@ z2QxQ8j~Mw)cm)o6Wb1vaQpc)Mov+?fu#6`k=k{qr-AjZkiMVJ53n}^h_sdXxd(h?w z2sYr+WB^%EIjsH{L;2MA^&>*EmiFM)Vx{5kjR(tRxH4Nd;F7H*z7)|SOkwfFzHwu zjyQb=&j52A)7Pv%WBN0lYwa?HW( zarYIRMDIjG1c=2P^h@kN8wb9$KO$>cd);0G+mrbMnewtor?uCd4zr1?o4SR(Cg_nF zkUVpjEWC662=|IJO^(DK?x!B2j^0b6y`ZUikbi4jw#7kK+WJ;36wlKtOhDmSF}g!P zH$f8Obzo73Pm&!K+EXYKY3-PIga1nMPS3b{_VSCxM!!Teo>DGkDn2nojggH_aaXpU zH$LH4G7AvW8Fsi#Kl+Y|_v>v#Dx3|0kueJzq0pCt13sdNMIxa77x~y1i2v)r{k7Il zND=PDKSinZ=X(DiMApBZ_Ma!ai_Pbtqt7uICjU6<|9QkG#Z5_pF`)&^zp@lGHEY?> zob_KdszR+K1%w7Lw$>K?cE%}=OA}#cIkRe`ZoT>9P01uFjZPP!xp|Pi`5TCO_viK% zatD421$LF$U_%rr8raKq98kg+@S5i*PgsVji0t{U;(+WL0{{<}<}w9W&4F{x7$Pl( zbVjo%I-2ko6E1HZJ2oWFx(V7DOrd@d5*` z$^w|?A0wYs&yY_5+5hSUlD7Sv!}ZT&=${JY-yXt14J&uFLv-IB;{;7|FhDTKPqmWt zSYs!|FA=ki#QBAQ@3=CNjq3((GLRfB8)<9c7ei9omq5K)b~Ud$=ylJZQYb>5i+Z*B zzn1gmKOQ|^d3|2DbE^2vOkEo?fVE))zmGHSv#vM0oqkVW4Si($efiF_l~f1eR-H>m zT!3nXB7;q=5h;0h=b>ki`3J>AN^bpM0t?`p#cS6RwMP{$3 zf5YpWx9YH`H(#WeiS>XF^CvIoZmh-lGlgi-e+{_h8{Yi!^Lihu^G%^Y=J}bEP=IJ3 ziy-mF2-~H${~5-Q!o!0yGUx8e06S*KOy>7J>>KEl^fOt7hf06Vr(Ld4S!ktk$I!o38En7`E1Gm*Q%|uEZ}mZi#3G(m+FYjfWOvb#E__Z- z@(F2jsA4hk!V}px!d;J|`8r=x_bYB$J1ll8Ve2jmxjaLGP=xs4h~P@22`UkOuxu$Wok0kqIKZY z7v{k&6b-io3)$3e)lbt}Z_~fO@<4lyx#RLPgJs(e+HrP2uvELL0%L{k5$EjyTCw_+ ze))JVuA@XZtJY(UUm@n~7IQSIt*5nZU*U$mCqwT1D3z8{4@xE$)w^EVdEw%s?L;?S z2|Izmbm=3AO)bgLcxt0pe<;d7iUbn__kCyk1MyTdvmGeC8&7J}*?2$SH-VwY?XfoQ zqYq<9sSl<$9(~Gzm3ef2XRgI*4*Qs^-iOc{mq!7#_VR<*pu?P9q(kH6HYv$S-z zs5+&xfW0mB?!dqQgwI_$GFrdABguYE1tGtW{WA+#TpmLPEWl=W5pr5Aw28uX;H%y; zShKW7UZO*hqu~tPrf!AcpmE0#hMhK)=uhlz&ZpnD;{0lol*8qGnCqv4KG-m|@U3b4 zl?Xtc+Q$Xc06T3Y*XQIe)_)JH*Bb%rJJNTuo*~r=xnwEUULbeU+->~Au=|AaCwC|2 zB2ZvgDEj*SPVt7h<9)_mv`@5?y4II?s5&t&^ql?IDp)~#`3~D#aLNhVyyKm%5s< zL&hyL{c)Pza*a=VISCsC*4=Tp=uQtG2X4&bwB(@~G@9X0<6#s;1YfU8ceDfM>1HP@ zBy+SOJ62?4+$-QM&((;$^prmdeX|xh66wMGkkKp(BhH)FWT^D$W>Xk4W6R9ON7z@% zVRY~IWw$uox{cpl(D*&pG(%7Kh}q0AA?!&l`stAjp6n;qfF)3#d6PKfdL*zVS>}wq z$k(Z_&pyty-RR4EK022?p=*_MgQjvXnxjs<8w|nxC^pBs8a0=({ZRGv?T7#LmUdbo z5r#Hl!!S!-O;@J)d>IFHl5f20y!@7DrF^ z_z?7%h9EOc?w4A3|Xj1uxkVSJ`K>#xa_TW471++r*d$j(!vspAXkqQ`^ zoZD|(BIil5=7YRnGxHTWM533mjo-NK;i4vPo| zn?ra*s~AYnANQ59amix31-PTaLuxI)j@uiXoOo2?&_`tlv)X@EpUk}*8n&pEBNV@j zfuXm{97;!lkNX6+v9zp?5+KP7qR1!wDyh@V6iji)=-R7v3v}szoJ4rNwSbSc0fn-6 z(%(nYl;3B!l$3VIQB5c{^=Z(cU<^xg(h!)O2HJD0|Dx{h)-S(T7;?(y9>PQ`{RE6a zN^$cR<(fZBWlxS$nRU8TMF?c>_K?{Wy8Giv%3+vXj7lz=Xr6dgwdby=-Tc90f7Rmn z^RwmZ}Z^0gA} zpDV7;h)ustN7vN)e6%NhRNEoJ_;&(`T^2pSMT}J*hGEJ`LU99Th$e4vERN_Cl7g#1fFt#NO)B_sw+ciHwEO-DTLU;d;C;dye z;aOqSO?_&%jn68b<9}bH{JlC!Qnis^P(bswU71tl#@~x8%r(>Y)Y<4)?fVX?&>J1e zdSv`?r;hj;u^UCIqcnnP3%jix85v47(NBMO6yqQ~NFgO$8%!~D(h#Rsy^ znD7kV2Yq1PEh7pH_)>|VT8Le`$~LvRf){u#Ty)~YrURJVsxwRhFE305zP=MK(trR9WN;qn5XYoiV7$sVI!-sf{?Fc5ip&cf@ zGudnOY3pS~zMc(WnAgRHe8MTkv!F<~v{cir@{BfEiuf5tq};~7zttC~cORaG;|u%A z8FeY29j1eTQTh7pT&GEc=CMS64QhN+jpHJHpUzv5^nu@Wie3C#1CM*Fx&F-9i<)v5-{o*hh{<{gKx zd=UIIR`8%zBdo!+hVyj&mt4`*Z~b^{sH4B1EheAOX$fYz1wCYxsp=EgFdIn{NTQ?! z>J?uF%QS{ge*9-;b6~hcvHQ70Y(I6h!2f24sM=cn|ETCL)d{7~I-ZYj?3)4ecW9*B z;84mjBN46j zdtzrh_wdA5IPhahW^WKPnw#v7R@qkTkoZGzAsT?)f=Pb)y67CDIEV%IE>TE6pUgTg zL9ZdRW$gY5xw!Ci462Hx`Qj`7ql{#Fy#ut?rfFT}i$MKVFUxodF!_t)=KFcy*2A>u z_moV!M0=Ff>}`{iEgwBS$0#Yu7Ct48thL-)BjwNg@UxAr_*gIIEQ=QTa3jL`EsI>+ zjYuZFeNxBqt$JrBLk_Ion==8{S`XJPyZS~xDjB_0DuZHdF|98P6U&+5+OyYb`R`>$ zT&R0d)Shz`lbzwV-&7um%7g?{!VTcX>T>tUF$!9T7@uYq`WTncHb52^h>~r@jbC@! zF~bd5ftDhUhb}_yZU8OJ2*Xn2&1ia$Rn;Sm7sb%p)Pb+^?Qaz*iRYMB z{HOnwh5+*Sxe|YKy#HBs{>$%G({@@=L;915HBl#u0zbG*K;&xLx*mU+% zHqg$khwqNi@8cEMKjqd2kB5=Wh&!5H#zmiJ>!cUnH29&|eUi)`>>g%wknYx9dUyk7 z6F_YwD8?T>eO49_%SzluTA1%nJDfXWFHX4~)rVA0=CF*0C@m&A?1f@nad48XnLNwp zE8bOcL>+B=P|RwkAvBVx$EsRlXod2)+=E|ci?Bp{RE($15ivmFA*6U|Xd=mybqYmB zu#QxV!?x&{MxKG=LW=FctT;RFq3zZuAprm?YtBkFC#wgKRa#~&N`;=AcGrK;T9x2{ zZa3FXnyoEKCn}04|aPM{QB&kb#R?2`EaZAdyVs z4NDNhxCE(jNZ3uV>6F;=aw^FftX^83sJ&I4mXoE;^pDxhmL3*Or|-UBbQG$UN9wWJ z_Gr&8`m-^ODr@p$q=n>t?3;VRW@%$E@F%O+)E0^L7Bk0GL2V=ugD4wo6jw;B4HfBf z9$QHbDDg?Ad$rCSd7b5Atz^Z2FIX`vgL0-hQrrvRe!D7O@;)!aZEA8H(l^x>vYSzH zF35#}be?z1YA86ZtzpZPL0PsKsJ()yYc7G}B+!0WSBI8hz}D*h*3E68CD?Du(Q-3L zwfLiSYE_|iz$(4oxcw(>fV}v^$3TkoAXSCoanpT!u9y7iS?o>u4w3bIBgSsNmYy*h z1W%o`nC8#GF4)IVaWQjh*sHzl$ltI8`Z9gLpT7&a8Oa-am+p{yPkvq2i8(iE=2^<$ zfPFOR36Q@j3jh;(h0lq&#C%}-VI{P!Psq@u4Lgsezk6)uhVSyjP^C}(S*wqO#C1D@ zS~bMmaW^{IO-E_6a?kb0qz1PaVOn;FXEYz|4+s>)b~lTLRcW%R!an^43kyD5MPZhRfWFGU)S%1LxsUp2 zE;Us$5)4>8;;&AzRRzfgNC~-xa1+bPl^r;)_xBxv+pm=I0VetdtyTryvPe_gPxm>Q z_J?3wcRi!xZ1W~0RUF-rRWG7hZKtOXPpeMOs9HYA}f$?TK=>Aff z@~*63NyAagJAJClb{y!OE`fH6#4%WiTMHeu%x(UjE6ke>r+;+=D#;049D6mD``YoG z&czj|I5U=`*T2rGnx~>8-_;gEt>-kGvt!5mgZR9IqqNN_j4}M1cGl@?_vG#0PW6A?WBxxZ#Uv%&zmm6n4Vn$Eva*fodv1|bVpO(hKfaNee@6>|jpKc0 zmjqmnnyQ&nkbM;N6v_wlKZCs%guAhSL58z%C790YIL@BnbNlo5{tmx`^uyb@_HUue}m7G$vd zI5w&QbjX4RGH=e-Y&Izgagbzx3!ogC(g+bO^w;IE3jJuu$#Z`eX#9;(yw!S zpqe!4ErF|esQfMRrgi`X06a6k9B2myu~agt{0Whb(z^ZbHdNFoP1U`Rw+iEw`Z*Lh zc-Q<@l@U!(xBZegoF3uD1t$e-JVD%mQkJBR%(I{VA|=?!j6|KU9U{n>$@I~$??v)w z;98=w&&N+hv|};w=V$2FMt0i5)=+sWdIPSZ{3%RYClgv0faav`D_>A_k9tv--8&ZBr9J#qbF7|M*D%$3JfTZ-)QB{C)K` zKU{V6KVloB^CESsBhf%|)dkLE6E48FS@*JteR5g64Jrpodo5U0D-!uwv&6a4Fl{O6 zyxHKQB8c>4AC`!pg{&~-0?8cDx6`0ER% z(2ZLil$_5P#|*_SXgAVG1ao)lP8|c;Y;=mTH`h;FEC+AV&L7eH=(oPA=AGnnzHDJN zyY(8rx**=d!{-j2ao-WU(*`r#rBA%2dAsQd?8cs>gDy?imSqbZjXjo|oeM^@$|BC< zxiy-8&F?g75yZ_hS70J6RcoaOB}DxY2bxH-g$L62jwV{5Lq#NOQCbAvllj~@ER~xF z!#Fu8vcW31=Vdw4Mn@uZsWvx;o337|70o?Ynkpam4QGJ_evK`nOPAK_gOajPx0c3l z311T3b1G>1Y9>_!ebR;7%-$Crbq27Fkvy4Prpl~klji#|FZMbu5h{}kA?rSKV_F_@ z%ytfQD=XwVo~bTvrXr1LW!Ertm`NTalpqU3>{Ao;$bF4x)NFB^q2&7P43-oNqy|O~ zJWOA#xsyxq7&4qBfdI+9&&Exo9v_JS-<(9XP-HJzQOdHAaG)7)hIn4(~N< z(mX#}|4=|llvZ6>9**iSEQ8ArOdxb-R=bTgQ7?tQs&H+yls>WKwl6GCflZgFs#K3Y zRSVXd=Zag)y#b|hcAjYe`KOt-G)ar^a+KZbVJ>4CVyU8S!L}rhhya%(*@~j0-UWik zAX#d?bD)K@tIHUpq>-%Z;K@w9wmwK_Cqx#M5V{!~@3jp<mvau-CC&*inT)s~J(4bQwef5NKwgT8Q!5-TWq+{Q14#0wUy=9Y&Ypv{g%UCpcdp)X5E#mp<_EW} z=2<&j9^Fa z3EXedVfxY5Pe70wdS^%p+8aT|)B!sa1#u?2OPp=CV&*`*TT`71PI~dxxHQMLqlV+_ zWUMw+Ruw)|rszJSTi7>|fUNVG>}E*etC>g}1Y86wHL@tqh2Qqg*qt5}vw^D+3tB2+ zOlY&s^1h6tuT%NS6X}W?gx*ns*7yHkXhnJ+UWZ{Ae>r|Qbk_)fB06SPL%>VMWpwm+x zP?NliTb!q@+a~(@Q&ktpE>{)L@>Q;Wgg8=R663fb>J2Q!78L#C@8y9LY2?-4vk z_VD8&5wvW2OdCi?KW4s%6jHk~mEnZx$`Hrv=UxVFN3|Te!P+28e*l_yVLT?&m0tzF zmVCLhb12&m4(bWEq&P>sQiougHCV+$&>Ci9;0S1h?h-E;Zi(Cvg;uYtg2|5aE7~sA zlE;v1(-5Hja}8O%R<7?(FE7NU;?vf~)EeP)K62hF4l^F`dlP53jXB(ANaPMf(LA zfmVcBTft7P)(yoHv{zG1JXx74PpyZbHDuCh3Mb>T=t*v!oOq@r?DC-d_s}%(rFs^^ zZ%d6I;6Giz^0sukzltv_A#7%@4AKf&qE=|ox(x0C`?lw{Z^gvhO?*eU;u_V^d6E9W zaC>^PN#!|DT>$3Zp&}#PJ}eydoid0XvIwPB3pE)2kBgEmre!NXABiU*HxMV7`Uz$yH)SRse@1O zoS+s^kzqW=Bmk$B17sAN(<_%zh&u=%w9Bp3=QzeRq`5~r^rb{D=12B3v5=xPUc1&8 zCLh@^od>R{eDUR3l*po`zs;uh_X)JBHCGHG)wgq$QMZp10IY zY2=XbVB_DLnONUkW5OQ*=0V-wY%*MU)o$!<{o_(bY2-BxVcMdMSYM5)5F15&N}G z|I&m%7V5&f_J#mGGD9nSQwLKg!{7TN=Bk*{ZvfixElr(VcAT1vR{dY%M0Tljq9WV}=Z$6IiTT;#oFAJFftdiMJ#wEMeD@ z51w^Pmnw7ll|jVNT-J<1oIp)}pY^abX z$(&%+DJ@cldZ?NZnM|!;*O=buWpVUW))kWmtz4NgS%^jN&Ywi!&&4)DLbp!JEbFytdRjmC-honQklfu%p{-bM^z#od$M*bQzSw5j zX3^Lp$w2swMHoF}`Z9WIzG8}u;B1R!=LYOf1}#jITe~~nQ;Uw&c+fa3Lgwn6)nstW z)1$tR#Q}yfd>bWF+Ob&g+BT(j=L_^e7|fj#j6cl=k0M`f(ftD-n>(#4Nl@}rN~tYS`Skee3!%|*_nJ(j6D=FB{b~2vk%t2&mxW2z+zjLWk+q27P_>9h-);iNanhX7PBDbcujL#~5 z&?(YirpKTZD!9x6l*G3Iph2-*)mo%M7O15Y?K^}Ss$rFWCN3IIg9>mn9T@G zzu)<~9Bo<%^_osbyA{t=qjdLDcdWLqyImc2r20f(BX$ETg5YLvVam~48XGJG7W5fN zeWs9lWApcxW-#tWmXVqY?yLp20j9M$T>My(y|^v)LKO%s%ODWLrLz*#LpXDdEQUUd z+D6kuG2t%RG2}8P=+solX{al2{qh6tDmV_>7 zZ>CY~Nt`JNL&QdxG8121=3=w47?gNkfrjE=GvVnB4cFHzeYMiDr zF5_|pv(dl>#!gc}Y4?VQ?1O6TUQNUqR0*hUlY5Q4E~dlc>lW+{v_{@P5fZYFa1c#X z=!TAp#_q{|jfLfaxnemk*gTcZFm4I4BIwKCaX7oX z6_wEK72gdI5+u7z4z_teh3zKKJ{7F9Y3tIuRr_Ri@Bvf7W=@p9;Bjmp&Ktwe1go@g z{G?eWDaFAruknsF)IB5n9^l#&E2dmJK4$G4!9U`y0i_Q|(feC(gNFEr!T9rk;$TvS zqh@V8#8Bth!us%G-sNs{bO&Os3lgsVUB}zcmS08vt@n65^4Xp|reP<{!%*rXNk~aK)LSlNoJNI(_qdG2Jodh6AlOv+{ z;JW)ApZ^)A@XJnTtgl4;2A{h<2G$v-cp><^qRl*d)+?YIEFckv*!>B&DkwlnP&>%I z+A}uR>lB9l{8NnJYkY7UV&6}?r1g&u1J2qohUe>(H(k~rmv5R|-%=k|EDZVRiDG+tP<5~o z(OJW^xW0D_;s0>w3YAjcdIyc7-{DI_UM2>*Sf4_S)}oEi zS*jSwX~y7w)XP%PZ-fS^-)jT~H9o1m>_H)Z8@SZ+!=i9q?d)pND0RQcBS}y7^0)A!t86RLC=*8Zt!3gCklci zCWm;JEelpqZpMib)>0@$T6o&hMwL;38r(T`wNj0t*bXH`>4`_=8;)Ce1p}eMf_!&& zM;&EckhL~k#kMf|v|%ZDuXyBE#h?gsErc=rIFo@re{kVy&K2n{o6VKG3}4gh!VYnG z$T0Z}F(zjh8}9uHtKO4jLFt|bESh#3O@;GbEg3M*LBwNyi14t_fM>W zCHt(icYV<*C*4#5LG4lqo|u{WdFTkUAXiF^wuX?kMivYwG;cUv&>EMm`fy5DY_GEY znMK9$!aj|hES+Z<83-j~skhCI;Pu%H%+>^Bz%{6~ z>}YpJ(w89-R@khjmJ>;l(@rr{&Ut7oxQ4ROI#n&FwiPsAf@tHpW)*0}`|%SE1dzds@`VuDdk_DPCuFR*g))jsMvKP(mSkc zWo>iNM5cWmg-V{(>-AG8A$LSX1>5fmn2oSeZn%(_a0~7T8&|LOPYoH6-pY3oV-UAG z>$w!2>A-)ojaB{TUre9i&ZqFm*+jc}1Bec>7Hc z)^d17RH6xWWS;#2-z8jH`e`9#2wDjwYAJiX_=FtYQNC8?RGJ(08CKIcTk6nC8|x<4 zInb7n1cgS>8AxTxduWGD6vjTII3%LBmVM%od2z(4Bqn3?jQ1bxhKHi31_*Gjl)}nX z4;pA9m94^*U+@Y=>peo zNo?*$Q9)7N$K6y$*p)D{cc%Zu`DTK%K-lB8n^%GL_$`in;M$b1F~KI~*UTFnhF5T3{g*2a#kgwo$-jYFM0vcd$qn9xB+JyI$&7jpv=+%4#Xh0b<3g`3 zhdF_vA*~t{>b+!2>!`@0Elcl!sxDEbBBGAVD}8c_0)Eg^8|ej6d_yHd_!=@JmoYjU zDmq^-Q-Y^4Iaf={Hd9=CS%Sxv^kma=dNjj<{4BrZ7U*O{+?0VoYOUXdkY9f8yred3 ztpN8Hv^myWp5ub>K;@BkrH^pxge&e5<$+hC(#oJ)^}zggRWkm-)(xahJjQ~#K#-+7 zGw=m_285~N{?R4b0iN&}e;|6(xLHZ+m7tYXuBueU)KvCVwtw4faK<;_SvQmp^E)m# z2wuc@7d(|cOp;f4&di7t?qK&Dvz;BHR`}wHayxGn2zepXZYY_=6nL{@%h-F`d>*0Q zn0&eInMklJ-Zc>ySO7s;tT*FI;YoqMpa<82%uwHk)3aqzTi)x`BaHh-o(A5M(-c%C z>xbJ`o88`;vnf)yqQ#QGGlmJtz6c!pA6!rAUmxa%Af&ZuA6pl#o} zi;9zZ1L7ofenGj%Fj-a-nA*sqtn6mXc1h9sIIkyAemj!SNmgZ#+E!dS6+GY#zxqki zb%?-K82eoIzTOgfj^;CH>KDy#wY5d9Nn_w1vmh!-Y{1}0ZU?PFVQB5w6L!hDS!2Nm zFFx3+QkhQ!b)#H!Qnv*Nwz8+e29Cm1DoBJxbWja$>oKZ_j?7xTXnI@*OLCl)7N~te z^!mihr{LC84r@yM>a9}%1)_{R)=SSrg^r5K4bCz@v=5aU^X*cuVK=&yFmrD^Pd;58 zc{+qj?;Dd4ZD*S#@P<3-KttrOkR6cMPKCYnlb`7~wXB2OAeMfC-YoxO7+49pwP7q54W_vapya&dxE)%o2JOs4eiSpYf}8%s{8$4xxWR-`$?m$BIN3E zQoj|Q14<@0uqS?Ca)WJUljz837q%vSxmIWj)I!Phz!PwNGaWWFenr)_BF$zXoB&x3 z5AKB{|Ju|RZhRi1@rkc3yozo8Lm%0PcvIh1fczcQot9UMmk7TY%I%%{Jv6J$ghe~# z;;NX{@iF*)kraz$Dwl~RuXp0)#U7Bd<}3rF(S-|X?-p-f{tJfT(EGh4lN?CxX?fMo z6r(nG$vg9Hdaau3(&+jEObS+Bh2&`$3W9=Vps}yJSHN7y8hLvl)nB19>mWMwUr(xB z`cjbCljUKIR{#?{A!NG}3S4ASfj_~%Te3Fc^2~0n0$T)oW%Z5r8}9Y4I6v?`CyjTc zW1ibtcRdK^@IHShDdckCr;LN_ihXbY*`Rwwplb1h2Iv^eU_*mB%Aq4r%EZRv717gB z#EEi&Sa+vLapDq6h49;lqPn6=`dFaMzOCt1X+b5q=+_c-Qxn1)-O#BL z?4w>BKo5Oh8p~w)#te?LkZ8AgTQ_-ON;EC+bWR0OfLCGR@EB%TLDu*uOzv5Tz6qzw zd;41Q!&&Wk=*46P=F>+7yRpw?PlVqlbeRx^nUAjur^`NQj3Pzy`#W8}LQ^~C z6SA6PQBuL;kvli0x~wfGh6oEV+a7p%`B+Nk8B&T7fhVfnSP`m+nY$9nqsg%d@Wc~# zM_=SMx$)~a9@i=R+US4X?quG8RXz`Q(BSkHT zzZL%YCBF43|FhH0ZxHz=-vbP*)PRj}mVaTYA`>(M#1r_#4Z{8pyH(k{|M*PzKQ-VU z?L**Fg?xhb6M#*~$qivr@PHZ=L3IdiWpXyhpDAZ^7Ck4u)%G9`mC6kz>_jwt*--Mn zI@XU#*6kD5&GH)8-m3jlR!93cpjUli5a?4l)yW5xlB!H4#F4J>6c|bg%5==zhh8;< zyd|41?AFey>+YZ$aGVh4n4mH6TB)adN zYK+1QTX*9O8#m{gG5r@PhWvu{NdH>wcM?lXya3nbw$IihoXQR|ZpYSOOQuu&(b}=T@mEqbXCPJXqLOIyr3b!WAfOaIeN;` z-7LGUIB@FC#~qNa$tcN zX5}GhmZSvke@P$`rZPL;e>k2T2(loybV{`eh9reDyw4RH3)L+0{_^!}LY`8%ZSv(X zrUV3G;^Cqgrl1<+E#$K7eJT&}`=%UkF)5?@n**RqY!GS7IfP0i;CzQDvcT<&My#AO zF*fr!GY?U*wL&wyAZP=1+Qi~nVslG71q?od^V8Vqh^&c7-pUnF?Di1DTMGeFr%jIJnq3Q=CtEJBZ_*;5gE(xykw7*}hyR9nX9@#V3r8;{bg= z9k7}3uloES-TjX}gdbiYiYId4y`QHL>l5m+*$u)umJtKqa7I!B zrxZZFnHOBz$eMEvzmLF>{uWRmRIz35G5{1bhdJrWmsOl9dOYRFs%(mZ4TJfS3GUoY z8%!{#xmMt_#(CV~1B|S-+k=;OYwgZ>zdOx{bV!sg7PlQ%>~3rn9h{y{k%j&?l)QCzmpt zHxxOZhMRAamdjbDet~6M;}H_T<)05-)m@PTUhZTgox1US@22rBt{ z-J$qMz3nMW5lg-gSZYiiq_NTjy6dn7UP4QJVp2i=3C6W}#NW{4pV))OEIfENSc%N5 zbbE!*rXyV#@6d8HG0N;1ObGGvQsuc!K@Wsp*` zj5Feo3Uo)L`d(V)_%z0LSuP*wifuNhm>MU`mP|5@&}jDLHOFDCobqak;&7M-#@@sJ zeFwsYVpHiYKvWb576^#v*J1t7STX-*6BG~_&>2S=?GXV^hB;-|FNckyOcD+q1iq}^ zl%7>;YWnSDtZ(>_2##64LRp>! z*9*pH*4_{4p0`^SZ@eLG!1*OOgO^AW$vzk|~XXb*oA8s&FrVSWL}fi3lc5@!SyzO+s%&`?Oe4 zkR;*QmG35AhhA2SfnDHMst&Gbu8v2CbziPhoC3y$#s^&wx>%s65fF~LQNgVrv@mxR zUq4}YMRP*IJc$7-3xj3VGR{d6j^BbJ%WPJ!5>-JxBeLC-&aBEt1x+U_>6S}iMZJ^S z{JJ8w*pMp*q;3Av#JW3}di-p3a0qT$C*YE=iO|fgLuudeK7z#s& zNv`(!Jnv6-HodjGbW|B!i*4|(Ys5zBJ~*>wcA0o%dipNK0kmzf#N-#>oxdnB9C@qJ zqmO+ij2xS!AfV7k_eP~hiZHVz>MJGKryBS=Jt9|x%z6QXsaUE)u?4@0oV~6XtKPH!#1vpK* z)jy-Us#vP8CUT(F$k5$gTT-(9BIt=Sr2MEW=D5G!{LNB#?;~qb<#@@alHw*Y(rGCW z_UNXnRikDXV|HCJ)w{NGc>Ec*^~BeMOa69Pq~D7xb5A}Z)SD|_#A(FgMFKXqvAD$jkUTW(k@wWxE*Am-Wev1Tj7 zx-)GB`2%V-{OznB{Oz0`D>^RL{NtQC5J+7-m_u0m9x_QZ-baTi4OCwin2G4vRVd|b z#uKAfml%wjM@rWl#jJ3<4tO|rB7EHt-EWh~20>YsLjEjp3>!_lsi{&A`x9vIP&8bv0rcuU>SbE>yjv-bF98a?n5Z z<&ufd_I~;=Do5YX@x^DfqPGI(W+K4cO!w=+^@~&HUr$>^Dr){Gv`Bpe{Rv6F7MdUx zW)A&O7-s9M5-=uW)1EvCe`{eL^`P&QGCi_4%qNQW+w4%O7!gmvmj0*X7x`$C9w`}I z4;g13Yg{ZHPaaReZ;@%K4PL!ckRkRHLcC|p%H0VHOOvF_i}A_qqo<>=)$}_1&`8^H zvt6CH*&YT7gW^77VXG=wP(6H|y!s`eNHySyo_4s*1 z=f!O8)JUBi;6FWAAFRSw+n8{(KDOC1urUIu`~PH6oZe|(s}4- zFv%F83l3UnrivGQB0Y$6A3I}%UjsuSS5lrv!I7+fmE1qKI@|b3G{lEhwV^4k#)6@t z``RT+{ekO=nQlNxk%qv`1uX6*PM4%>`2;iX1goo$1GGk$e)G!}Iktpzk#0q5lk2BG z*C0cE_oTTG!AWCLL8l?KtKe;#h3|rB#j94i?up(xh?~CE_J?^IY%l53HfL?~l$lH$ z9QV)l9egqA>?4N^MgwjDis2eFsWsyx@54jl)hHan_28D?XK6RS)qj$rJ24+Lc?&)l__&M~~Eks$7NHRVjM8kB= zM2*Q*(ao%}v5^$}0_WY+$FEo4Abng*DPwxz;km27m4?S>tJ|l^o=!_A>N7_qX-!bEf>kUTDZId`-pC}2>~VC`+6y6aEnRr{w%^6Ww_2d$q?v1ErT)c*nRj2k zD4R5}HmXK#oGFBb+e%-lH#dr$u0(C3DvT{C@YPPFaXWQA-NxOH3Y!EA+zqUB9S+RLG+TB1#P43ER_cn=mVZa!pWn!m&=jF`ZEUFd8K z)@|48>!9(1S2XvPNpz4&_1*w2GKmzsrOcT>_AxDqm>1!MydeFD#4mJ+k)uStModCg za62D@jzzP-k+G5VxI(`NzJR$yL1!uG5V#xP2p23@t_r7|-3Z-aE`OD|LDL74$qQGj zAiBkpE5as_=qY@R&kXs>IiHec+Vv~%Ye

i1O>I_eGKr^&y;^hm-V94&lC5U~@}s zTa5yXP^(8uV=BE6Dt+3? zLaE_7J031uGhDE7|b&w1(^tMyMTy4Ezl&wqN)pF3smtPe#|`O&uE&FOFx;Lv>2r%dWXcsINN|d3 zzSwY&aG}}M#$$|eJ8DI()+D|nV{jSt3X)A(IZ~?lt!A^rpo2z1YmxIU1%lU1G1LL~ z*NFxe{FkBI2+QE9yNI%iZ4g#m1ZMFxYZ10u>`lpV;=aIeh-*?bt~GKz2fOafq^Y$g zM%cM+$NDj_M%^@usUdiHr=2peP3Y;}A%428!?wZ1t|ZvKyDc#F=ivQGFk$n$Auf_1 z)(P76l5Zj$=3GpYqa$Aox}1I>2ghWjQ1cu{;^*$HRyLhx?bh*pJ)4^KC^Ti5mx3`! zvq54tCQ8wFggOQH(f{j65`l?f+7=HujTDJq%0}^CEPFKPxXlgtM$%#TJB|{_HY7WM zd$9`r31iNC3?q53j|V!gg-2DPTYgyV?leMeIyCC^Np)_#+#JG5N3RiYT!$iODzD5A z6YnL^J_$q67@&_DHO|XTjV7qNFUdl}?SlKC>tG$*#ANykA3n~@zH4~pwrK89kqYs~ zSO`7+bxlDW-G#QxDvwT)N*#jp0`2Cv3kbvct-1AYh_@X*vgtjvB$wGRQ#=vi{<*DF z1Bh84t}k!gqK!CAPYZQId%Xi0FyxOvKCz5?x@0X*=|yS_6KV?iX!k=J;1;lT=_hz} zV=L5=<93TT&R^bhK#YsG)gu;R$yf4TTK0PRo?+jke){Nn3knK58FEQES$;Py4-j^G z(PxSif~@1(9Q{#mzxI-64DH)s7Yo=8Z4%`WBkwkcvPME0UF!K%y(a+o`}UBc@#`*3by8Z}nAp3?M6pH77=*ri%#`G z1QtkLqKP)tWpQF`2<_A6X&3Kj6}edvVlZF^2D?8dS3ObqWNwC zOIenicvYA65wda#o|w6m)lF2wmrdDsn*hSj7*D6e69S?j~z zk(0vUKo34jm&;RR0=U1()qpAHGe6dN0{eTD(xaRBK?0hy1z;><{QXNjG8ek|8)3)_iqnw}LZMK$v> zcP>Dw%~pt!=i@kesT1!3flkCj`90M=pS#q`TcWS1IWH~c(~TT<0|S<;Ugeay#fHqM zJQ3xWaJNGVCn0LC2t=mGq6n`!LbUHW_?@c_v%j(>0%{`@K^ll2$dZF9Bt}qbpR27XI#7 z;U+c(3^sbw9~!92lm}5O2ccjQ zWP+fnk`Y);H#FHdqQ1i4ozrAYV;T`diRyj;tH8UHfkT-Ix!6wNNo0_jkx3`w-eb56PVT}m! z1Lohml~_I04KHA|F$Dcv`{S?O>Ob2`|7Edp^m^(Q(xaRvR35Mtcn^=BVK#j&IC7;6 zwepMCRBECVZ*qEYsG#E^&e1)9)tfu>s&RL^@i2C- zY2oU4^msrZ3KB&g4Pl)`L7(sok*F4A2(ppxtn^vZVxbZ0O!nac=@WIal@ADc03m>q zV4O~~o)#h+8qb)F)FLw;C~+l^8!d^=Yp)_UFQHae#4vcDjb&R+pvpGW+Rr$1Wbs%L zNUlR)6%L3w?Gkri+%?_>exZp=8+NXwGZ^2KHCbclCCRST&H%%^tQ}O}W3I?Vq5S@> zQN2o74BTWy0ks%2111cNdL-vP7AHdR=A{uLeYB*kD;b<@b0U*rw~TdaaXGXEbM0cQ zCpxukm)k;`M*P}PY-{Nb3HH>hI;~c%$-QG&Z1%hL#7I1OEo=f$u9a%aLMb6>Ov9P> zHcM2@&oRJoN5)?^GP=XndpF!Lccdd2{Na2x#+X8|J%iMsr zBE*WMg3U}K_*z&RWY`E(;yLc7IZ3h+#;7TT>kAZAX&28F;W7`=bw*n`uO3woyC?-kF~99-?(=qf@bQg@*mw8t;f@}$ zFUn4>$|}JG@pYra1v+m$O1NV#np*C1*N1RlgXts+LJo3Eh+lI?hx_zS7?5Chqc@L= zee;2E38C=ITE7R8zok9Yy7j%kd(8TLCa8uGTR=SEG|^ZCX`8Zi^05UECSKxjO_luw zuW@tF=a@@l0k*cr5Nw`yI3wG0!j}hHOPr(WV-s#q6V*NyNF#JZ^!rN;Z5c(AuX%5K zM-cSO_ktIMpc`#~USvnFC-;Jvec-|rnjbRj2xRhs6``a89lhK}}SIMA??Kul-qE0}hn{(s5eKb02g3JKp&&xim%gn-Wv834dh_x}$G z07k`MB>;B*7Wf|+1Pv4*Tn+^Vgafef0eJr}>OEj|_;Dctod|yaDl4iaNG~ZT1~{ny z_hiPcx7GrHe^3T|&;Xx5uRP!HZ;StzOjb}%QcP4)>4mJ=Ul9kQyS`Xl-G zTdglXVEy$yuRoao1o%@H>d#2)&vVqf8aRsvnCEQ(CI8<1e_pk|z<|I06X5ez9bkRu z;wtFqU;>DJYpVYkP=lZx|*2EW;l!na}aSiriMT z0C`maq^AEHo-V*v{2M%RJ6!_{LuEktc?oMvLm6vpb4Q!sL*s2~FZBXI?g72@`-bqJ zSFJCg&;AY?5cAi<(H?NFOw826P*7jr(B58F*Gkv;Po;(Dc^`q-5&IrN`wn2w`+MFA z0Gg)1$-C+w${YYwhmonVBOrV>Ae8*?^RD2DZ@L4h5^O-;8UM^U;9l!{kMvvU=j|y0 z5R0(*ZyC>bWo%XfWE==kJKsl)Kd)L}dW_$q%UB!#*IF2}I|@Sw_?cJ%%KZIU)Sp+a zZzn(l@aFu)k;e_Z9J6 za`#W@THgeKhRkm&;zzaqx8dix#Nkgo2f%>-8@#_Y{eKfYd|n&R<@0{Bc&YxD<)25K z=c+kB;b!0b-{kpY0RHU+`q%t@-`skxdh-*|`0an!z@pBQGKad+*WUQO z-kwWv{6s}D{1>SIAjt7N|Ieikelnq%{0pYPItl?wXY*I#gXdJwBhCM$0y6(6s{ayq z{yF^fkm*0+6D|G;{zpsvQvmhnRL^6t{-kPj_!m@vRNDWluIJ$^f08UZ{|l0T4rckh zM9-uB{UpP3|0lA)?nXbyd>(7+C+4ZoKVkl8kUyaR`0*sP<1z(dhrp`TODO zxlip+%+ERh8}m0Fw$C&D+=1~Y!FJxiAo%Mj_4^V$cOUyn&|mlq!Jo6lzcKtiFUfOv zkDo+7rN0pU(p~@QaeKan{u4Q<;uqv!JJTP~zu!=QUWn&gmp`dotN)4Wf6S%NckF(` zF4X)J?2iHCANKE_7vQ;F|0ffE<1b9l-yi-`cmJb6&uz^=X+AgoLh~2c9|ij3_77v< zbED%=eDT(Qf&br4kk9MuxgF&v(F(vY@W-1uU_tnMtI8{BaKKyOcXb&iAS6KOxwoC) GfBiq \(.*\)$'` + if expr "$link" : '/.*' > /dev/null; then + PRG="$link" + else + PRG=`dirname "$PRG"`"/$link" + fi +done +SAVED="`pwd`" +cd "`dirname \"$PRG\"`/" >/dev/null +APP_HOME="`pwd -P`" +cd "$SAVED" >/dev/null + +CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar + +# Determine the Java command to use to start the JVM. +if [ -n "$JAVA_HOME" ] ; then + if [ -x "$JAVA_HOME/jre/sh/java" ] ; then + # IBM's JDK on AIX uses strange locations for the executables + JAVACMD="$JAVA_HOME/jre/sh/java" + else + JAVACMD="$JAVA_HOME/bin/java" + fi + if [ ! -x "$JAVACMD" ] ; then + die "ERROR: JAVA_HOME is set to an invalid directory: $JAVA_HOME + +Please set the JAVA_HOME variable in your environment to match the +location of your Java installation." + fi +else + JAVACMD="java" + which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. + +Please set the JAVA_HOME variable in your environment to match the +location of your Java installation." +fi + +# Increase the maximum file descriptors if we can. +if [ "$cygwin" = "false" -a "$darwin" = "false" ] ; then + MAX_FD_LIMIT=`ulimit -H -n` + if [ $? -eq 0 ] ; then + if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then + MAX_FD="$MAX_FD_LIMIT" + fi + ulimit -n $MAX_FD + if [ $? -ne 0 ] ; then + warn "Could not set maximum file descriptor limit: $MAX_FD" + fi + else + warn "Could not query maximum file descriptor limit: $MAX_FD_LIMIT" + fi +fi + +# For Darwin, add options to specify how the application appears in the dock +if $darwin; then + GRADLE_OPTS="$GRADLE_OPTS \"-Xdock:name=$APP_NAME\" \"-Xdock:icon=$APP_HOME/media/gradle.icns\"" +fi + +# For Cygwin, switch paths to Windows format before running java +if $cygwin ; then + APP_HOME=`cygpath --path --mixed "$APP_HOME"` + CLASSPATH=`cygpath --path --mixed "$CLASSPATH"` + JAVACMD=`cygpath --unix "$JAVACMD"` + + # We build the pattern for arguments to be converted via cygpath + ROOTDIRSRAW=`find -L / -maxdepth 1 -mindepth 1 -type d 2>/dev/null` + SEP="" + for dir in $ROOTDIRSRAW ; do + ROOTDIRS="$ROOTDIRS$SEP$dir" + SEP="|" + done + OURCYGPATTERN="(^($ROOTDIRS))" + # Add a user-defined pattern to the cygpath arguments + if [ "$GRADLE_CYGPATTERN" != "" ] ; then + OURCYGPATTERN="$OURCYGPATTERN|($GRADLE_CYGPATTERN)" + fi + # Now convert the arguments - kludge to limit ourselves to /bin/sh + i=0 + for arg in "$@" ; do + CHECK=`echo "$arg"|egrep -c "$OURCYGPATTERN" -` + CHECK2=`echo "$arg"|egrep -c "^-"` ### Determine if an option + + if [ $CHECK -ne 0 ] && [ $CHECK2 -eq 0 ] ; then ### Added a condition + eval `echo args$i`=`cygpath --path --ignore --mixed "$arg"` + else + eval `echo args$i`="\"$arg\"" + fi + i=$((i+1)) + done + case $i in + (0) set -- ;; + (1) set -- "$args0" ;; + (2) set -- "$args0" "$args1" ;; + (3) set -- "$args0" "$args1" "$args2" ;; + (4) set -- "$args0" "$args1" "$args2" "$args3" ;; + (5) set -- "$args0" "$args1" "$args2" "$args3" "$args4" ;; + (6) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" ;; + (7) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" ;; + (8) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" ;; + (9) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" "$args8" ;; + esac +fi + +# Split up the JVM_OPTS And GRADLE_OPTS values into an array, following the shell quoting and substitution rules +function splitJvmOpts() { + JVM_OPTS=("$@") +} +eval splitJvmOpts $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS +JVM_OPTS[${#JVM_OPTS[*]}]="-Dorg.gradle.appname=$APP_BASE_NAME" + +exec "$JAVACMD" "${JVM_OPTS[@]}" -classpath "$CLASSPATH" org.gradle.wrapper.GradleWrapperMain "$@" diff --git a/gradlew.bat b/gradlew.bat new file mode 100644 index 0000000000000..b5adeb2fde6e8 --- /dev/null +++ b/gradlew.bat @@ -0,0 +1,90 @@ +@if "%DEBUG%" == "" @echo off +@rem ########################################################################## +@rem +@rem Gradle startup script for Windows +@rem +@rem ########################################################################## + +@rem Set local scope for the variables with windows NT shell +if "%OS%"=="Windows_NT" setlocal + +@rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. +set DEFAULT_JVM_OPTS=-Xmx2g -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m + +set DIRNAME=%~dp0 +if "%DIRNAME%" == "" set DIRNAME=. +set APP_BASE_NAME=%~n0 +set APP_HOME=%DIRNAME% + +@rem Find java.exe +if defined JAVA_HOME goto findJavaFromJavaHome + +set JAVA_EXE=java.exe +%JAVA_EXE% -version >NUL 2>&1 +if "%ERRORLEVEL%" == "0" goto init + +echo. +echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. +echo. +echo Please set the JAVA_HOME variable in your environment to match the +echo location of your Java installation. + +goto fail + +:findJavaFromJavaHome +set JAVA_HOME=%JAVA_HOME:"=% +set JAVA_EXE=%JAVA_HOME%/bin/java.exe + +if exist "%JAVA_EXE%" goto init + +echo. +echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME% +echo. +echo Please set the JAVA_HOME variable in your environment to match the +echo location of your Java installation. + +goto fail + +:init +@rem Get command-line arguments, handling Windows variants + +if not "%OS%" == "Windows_NT" goto win9xME_args +if "%@eval[2+2]" == "4" goto 4NT_args + +:win9xME_args +@rem Slurp the command line arguments. +set CMD_LINE_ARGS= +set _SKIP=2 + +:win9xME_args_slurp +if "x%~1" == "x" goto execute + +set CMD_LINE_ARGS=%* +goto execute + +:4NT_args +@rem Get arguments from the 4NT Shell from JP Software +set CMD_LINE_ARGS=%$ + +:execute +@rem Setup the command line + +set CLASSPATH=%APP_HOME%\gradle\wrapper\gradle-wrapper.jar + +@rem Execute Gradle +"%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -classpath "%CLASSPATH%" org.gradle.wrapper.GradleWrapperMain %CMD_LINE_ARGS% + +:end +@rem End local scope for the variables with windows NT shell +if "%ERRORLEVEL%"=="0" goto mainEnd + +:fail +rem Set variable GRADLE_EXIT_CONSOLE if you need the _script_ return code instead of +rem the _cmd.exe /c_ return code! +if not "" == "%GRADLE_EXIT_CONSOLE%" exit 1 +exit /b 1 + +:mainEnd +if "%OS%"=="Windows_NT" endlocal + +:omega diff --git a/graphx/build.gradle b/graphx/build.gradle new file mode 100644 index 0000000000000..64ee2e856d38c --- /dev/null +++ b/graphx/build.gradle @@ -0,0 +1,30 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project GraphX' + +dependencies { + compile project(subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + + compile group: 'org.apache.xbean', name: 'xbean-asm5-shaded', version: '4.4' + compile group: 'com.google.guava', name: 'guava', version: guavaVersion + compile group: 'com.github.fommil.netlib', name: 'core', version: '1.1.2' + compile group: 'net.sourceforge.f2j', name: 'arpack_combined_all', version: '0.1' + + testCompile project(path: subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion, configuration: 'testOutput') +} diff --git a/launcher/build.gradle b/launcher/build.gradle new file mode 100644 index 0000000000000..22a32f5227a2a --- /dev/null +++ b/launcher/build.gradle @@ -0,0 +1,39 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project Launcher' + +dependencies { + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + + testCompile(group: 'org.apache.hadoop', name: 'hadoop-client', version: hadoopVersion) { + exclude(group: 'asm', module: 'asm') + exclude(group: 'org.codehaus.jackson', module: 'jackson-mapper-asl') + exclude(group: 'org.ow2.asm', module: 'asm') + exclude(group: 'org.jboss.netty', module: 'netty') + exclude(group: 'commons-logging', module: 'commons-logging') + exclude(group: 'org.mockito', module: 'mockito-all') + exclude(group: 'org.mortbay.jetty', module: 'servlet-api-2.5') + exclude(group: 'javax.servlet', module: 'servlet-api') + exclude(group: 'junit', module: 'junit') + exclude(group: 'com.google.guava', module: 'guava') + exclude(group: 'com.sun.jersey') + exclude(group: 'com.sun.jersey.jersey-test-framework') + exclude(group: 'com.sun.jersey.contribs') + } + testCompile group: 'org.slf4j', name: 'jul-to-slf4j', version: slf4jVersion +} diff --git a/mllib-local/build.gradle b/mllib-local/build.gradle new file mode 100644 index 0000000000000..c4183a09ba733 --- /dev/null +++ b/mllib-local/build.gradle @@ -0,0 +1,29 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project ML Local Library' + +dependencies { + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + compile(group: 'org.scalanlp', name: 'breeze_' + scalaBinaryVersion, version: '0.11.2') { + exclude(group: 'junit', module: 'junit') + exclude(group: 'org.apache.commons', module: 'commons-math3') + } + compile group: 'org.apache.commons', name: 'commons-math3', version: '3.4.1' + + testCompile group: 'org.mockito', name: 'mockito-core', version: '1.10.19' +} diff --git a/mllib/build.gradle b/mllib/build.gradle new file mode 100644 index 0000000000000..0bcbd130afece --- /dev/null +++ b/mllib/build.gradle @@ -0,0 +1,46 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project ML Library' + +dependencies { + compile project(subprojectBase + 'snappy-spark-mllib-local_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-streaming_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-sql_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-graphx_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + + compile(group: 'org.scalanlp', name: 'breeze_' + scalaBinaryVersion, version: '0.11.2') { + exclude(group: 'junit', module: 'junit') + exclude(group: 'org.apache.commons', module: 'commons-math3') + } + compile group: 'org.apache.commons', name: 'commons-math3', version: '3.4.1' + compile(group: 'org.jpmml', name: 'pmml-model', version: '1.2.15') { + exclude(group: 'org.jpmml', module: 'pmml-agent') + } + + testCompile project(path: subprojectBase + 'snappy-spark-mllib-local_' + scalaBinaryVersion, configuration: 'testOutput') + testCompile project(path: subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion, configuration: 'testOutput') + testCompile project(path: subprojectBase + 'snappy-spark-streaming_' + scalaBinaryVersion, configuration: 'testOutput') +} + +// TODO: netlib-lgpl profile + +// fix scala+java test ordering +sourceSets.test.scala.srcDir 'src/test/java' +sourceSets.test.java.srcDirs = [] diff --git a/repl/build.gradle b/repl/build.gradle new file mode 100644 index 0000000000000..1e51600f39b04 --- /dev/null +++ b/repl/build.gradle @@ -0,0 +1,43 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project REPL' + +dependencies { + compile project(subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-sql_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + + compile group: 'org.apache.xbean', name: 'xbean-asm5-shaded', version: '4.4' + compile group: 'org.scala-lang', name: 'scala-compiler', version: scalaVersion + compile group: 'org.slf4j', name: 'jul-to-slf4j', version: slf4jVersion + if (scalaBinaryVersion == '2.10') { + compile group: 'org.scala-lang', name: 'jline', version:scalaVersion + } + + runtime project(subprojectBase + 'snappy-spark-mllib_' + scalaBinaryVersion) + + testCompile project(path: subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion, configuration: 'testOutput') +} + +if (scalaBinaryVersion == '2.11') { + sourceSets.main.scala.srcDir 'scala-2.11/src/main/scala' + sourceSets.test.scala.srcDir 'scala-2.11/src/test/scala' +} else { + sourceSets.main.scala.srcDir 'scala-2.10/src/main/scala' + sourceSets.test.scala.srcDir 'scala-2.10/src/test/scala' +} diff --git a/settings.gradle b/settings.gradle new file mode 100644 index 0000000000000..70ea595ee61a5 --- /dev/null +++ b/settings.gradle @@ -0,0 +1,82 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +def scalaBinaryVersion = "2.11" +rootProject.name = 'snappy-spark' + +include ':snappy-spark-tags_' + scalaBinaryVersion +include ':snappy-spark-core_' + scalaBinaryVersion +include ':snappy-spark-graphx_' + scalaBinaryVersion +include ':snappy-spark-mllib_' + scalaBinaryVersion +include ':snappy-spark-mllib-local_' + scalaBinaryVersion +include ':snappy-spark-tools_' + scalaBinaryVersion +include ':snappy-spark-network-common_' + scalaBinaryVersion +include ':snappy-spark-network-shuffle_' + scalaBinaryVersion +include ':snappy-spark-network-yarn_' + scalaBinaryVersion +include ':snappy-spark-sketch_' + scalaBinaryVersion +include ':snappy-spark-yarn_' + scalaBinaryVersion +include ':snappy-spark-streaming_' + scalaBinaryVersion +include ':snappy-spark-catalyst_' + scalaBinaryVersion +include ':snappy-spark-sql_' + scalaBinaryVersion +include ':snappy-spark-hive_' + scalaBinaryVersion +include ':snappy-spark-hive-thriftserver_' + scalaBinaryVersion +include ':snappy-spark-unsafe_' + scalaBinaryVersion +include ':snappy-spark-assembly_' + scalaBinaryVersion +include ':snappy-spark-streaming-flume_' + scalaBinaryVersion +include ':snappy-spark-streaming-flume-sink_' + scalaBinaryVersion +include ':snappy-spark-streaming-kafka-0.8_' + scalaBinaryVersion +include ':snappy-spark-streaming-kafka-0.10_' + scalaBinaryVersion +include ':snappy-spark-examples_' + scalaBinaryVersion +include ':snappy-spark-repl_' + scalaBinaryVersion +include ':snappy-spark-launcher_' + scalaBinaryVersion +include ':snappy-spark-assembly_' + scalaBinaryVersion + +project(':snappy-spark-tags_' + scalaBinaryVersion).projectDir = "$rootDir/common/tags" as File +project(':snappy-spark-core_' + scalaBinaryVersion).projectDir = "$rootDir/core" as File +project(':snappy-spark-graphx_' + scalaBinaryVersion).projectDir = "$rootDir/graphx" as File +project(':snappy-spark-mllib_' + scalaBinaryVersion).projectDir = "$rootDir/mllib" as File +project(':snappy-spark-mllib-local_' + scalaBinaryVersion).projectDir = "$rootDir/mllib-local" as File +project(':snappy-spark-tools_' + scalaBinaryVersion).projectDir = "$rootDir/tools" as File +project(':snappy-spark-network-common_' + scalaBinaryVersion).projectDir = "$rootDir/common/network-common" as File +project(':snappy-spark-network-shuffle_' + scalaBinaryVersion).projectDir = "$rootDir/common/network-shuffle" as File +project(':snappy-spark-network-yarn_' + scalaBinaryVersion).projectDir = "$rootDir/common/network-yarn" as File +project(':snappy-spark-sketch_' + scalaBinaryVersion).projectDir = "$rootDir/common/sketch" as File +project(':snappy-spark-yarn_' + scalaBinaryVersion).projectDir = "$rootDir/yarn" as File +project(':snappy-spark-streaming_' + scalaBinaryVersion).projectDir = "$rootDir/streaming" as File +project(':snappy-spark-catalyst_' + scalaBinaryVersion).projectDir = "$rootDir/sql/catalyst" as File +project(':snappy-spark-sql_' + scalaBinaryVersion).projectDir = "$rootDir/sql/core" as File +project(':snappy-spark-hive_' + scalaBinaryVersion).projectDir = "$rootDir/sql/hive" as File +project(':snappy-spark-hive-thriftserver_' + scalaBinaryVersion).projectDir = "$rootDir/sql/hive-thriftserver" as File +project(':snappy-spark-unsafe_' + scalaBinaryVersion).projectDir = "$rootDir/common/unsafe" as File +project(':snappy-spark-assembly_' + scalaBinaryVersion).projectDir = "$rootDir/assembly" as File +project(':snappy-spark-streaming-flume_' + scalaBinaryVersion).projectDir = "$rootDir/external/flume" as File +project(':snappy-spark-streaming-flume-sink_' + scalaBinaryVersion).projectDir = "$rootDir/external/flume-sink" as File +project(':snappy-spark-streaming-kafka-0.8_' + scalaBinaryVersion).projectDir = "$rootDir/external/kafka-0-8" as File +project(':snappy-spark-streaming-kafka-0.10_' + scalaBinaryVersion).projectDir = "$rootDir/external/kafka-0-10" as File +project(':snappy-spark-examples_' + scalaBinaryVersion).projectDir = "$rootDir/examples" as File +project(':snappy-spark-repl_' + scalaBinaryVersion).projectDir = "$rootDir/repl" as File +project(':snappy-spark-launcher_' + scalaBinaryVersion).projectDir = "$rootDir/launcher" as File +project(':snappy-spark-assembly_' + scalaBinaryVersion).projectDir = "$rootDir/assembly" as File + +if (rootProject.hasProperty('docker')) { + include ':snappy-spark-docker-integration-tests_' + scalaBinaryVersion + project(':snappy-spark-docker-integration-tests_' + scalaBinaryVersion).projectDir = "$rootDir/external/docker-integration-tests" as File +} +if (rootProject.hasProperty('ganglia')) { + include ':snappy-spark-ganglia-lgpl_' + scalaBinaryVersion + project(':snappy-spark-ganglia-lgpl_' + scalaBinaryVersion).projectDir = "$rootDir/external/spark-ganglia-lgpl" as File +} diff --git a/sql/catalyst/build.gradle b/sql/catalyst/build.gradle new file mode 100644 index 0000000000000..d2deeb94b9b93 --- /dev/null +++ b/sql/catalyst/build.gradle @@ -0,0 +1,47 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project Catalyst' + +apply plugin: 'antlr' + +dependencies { + compile project(subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-unsafe_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + + compile group: 'org.scala-lang', name: 'scala-compiler', version: scalaVersion + compile group: 'org.scala-lang.modules', name: 'scala-parser-combinators_' + scalaBinaryVersion, version: '1.0.4' + compile group: 'org.codehaus.janino', name: 'janino', version: '2.7.8' + compile group: 'org.antlr', name: 'antlr4-runtime', version: antlrVersion + compile group: 'commons-codec', name: 'commons-codec', version: commonsCodecVersion + antlr group: 'org.antlr', name: 'antlr4', version: antlrVersion + + testCompile project(path: subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion, configuration: 'testOutput') +} + +compileScala.dependsOn generateGrammarSource + +sourceSets.main.antlr.srcDirs = [ 'src/main/antlr4' ] + +// add generated sources to scala compiler path (plugin adds it to java path) +sourceSets.main.scala.srcDir generateGrammarSource.outputDirectory +sourceSets.main.java.srcDirs = [] + +generateGrammarSource { + arguments += [ '-package', 'org.apache.spark.sql.catalyst.parser', '-visitor' ] +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index 741d2aaa3061d..2b2aa60f2538c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -54,6 +54,11 @@ class AnalysisException protected[sql] ( val cause: Option[Throwable] = None) extends Exception(message, cause.orNull) with Serializable { + def this(message: String, cause: Throwable) = { + this(message) + initCause(cause) + } + def withPosition(line: Option[Int], startPosition: Option[Int]): AnalysisException = { val newException = new AnalysisException(message, line, startPosition) newException.setStackTrace(getStackTrace) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 64b94f0a2c103..98153efd6710f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateSafeProjection, GenerateUnsafeProjection} -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types._ /** * A [[Projection]] that is calculated by calling the `eval` of each of the specified expressions. diff --git a/sql/core/build.gradle b/sql/core/build.gradle new file mode 100644 index 0000000000000..c6b3f5038db8d --- /dev/null +++ b/sql/core/build.gradle @@ -0,0 +1,43 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project SQL' + +dependencies { + compile project(subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-catalyst_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-sketch_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + + compile group: 'com.univocity', name: 'univocity-parsers', version: '2.1.1' + compile group: 'org.apache.parquet', name: 'parquet-column', version: parquetVersion + compile group: 'org.apache.parquet', name: 'parquet-hadoop', version: parquetVersion + compile group: 'org.eclipse.jetty', name: 'jetty-servlet', version: jettyVersion + compile group: 'com.fasterxml.jackson.core', name: 'jackson-databind', version: fasterXmlVersion + + testCompile project(path: subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion, configuration: 'testOutput') + testCompile project(path: subprojectBase + 'snappy-spark-catalyst_' + scalaBinaryVersion, configuration: 'testOutput') + testCompile group: 'com.h2database', name: 'h2', version: '1.4.183' + testCompile group: 'mysql', name: 'mysql-connector-java', version: '5.1.38' + testCompile group: 'org.postgresql', name: 'postgresql', version: '9.4.1207.jre7' + testCompile group: 'org.apache.parquet', name: 'parquet-avro', version: parquetVersion + testCompile group: 'org.apache.xbean', name: 'xbean-asm5-shaded', version: '4.4' +} + +// fix scala+java test ordering +sourceSets.test.scala.srcDirs 'src/test/java', 'src/test/gen-java' +sourceSets.test.java.srcDirs = [] diff --git a/sql/hive-thriftserver/build.gradle b/sql/hive-thriftserver/build.gradle new file mode 100644 index 0000000000000..6dd72cdbd08a1 --- /dev/null +++ b/sql/hive-thriftserver/build.gradle @@ -0,0 +1,85 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project Hive Thrift Server' + +dependencies { + compile project(subprojectBase + 'snappy-spark-hive_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + + compile group: 'com.google.guava', name: 'guava', version: guavaVersion + compile(group: 'org.spark-project.hive', name: 'hive-cli', version: hiveVersion) { + exclude(group: 'org.spark-project.hive', module: 'hive-common') + exclude(group: 'org.spark-project.hive', module: 'hive-exec') + exclude(group: 'org.spark-project.hive', module: 'hive-jdbc') + exclude(group: 'org.spark-project.hive', module: 'hive-metastore') + exclude(group: 'org.spark-project.hive', module: 'hive-serde') + exclude(group: 'org.spark-project.hive', module: 'hive-service') + exclude(group: 'org.spark-project.hive', module: 'hive-shims') + exclude(group: 'org.apache.thrift', module: 'libthrift') + exclude(group: 'org.slf4j', module: 'slf4j-api') + exclude(group: 'org.slf4j', module: 'slf4j-log4j12') + exclude(group: 'log4j', module: 'log4j') + exclude(group: 'commons-logging', module: 'commons-logging') + } + compile(group: 'org.spark-project.hive', name: 'hive-beeline', version: hiveVersion) { + exclude(group: 'org.spark-project.hive', module: 'hive-common') + exclude(group: 'org.spark-project.hive', module: 'hive-exec') + exclude(group: 'org.spark-project.hive', module: 'hive-jdbc') + exclude(group: 'org.spark-project.hive', module: 'hive-metastore') + exclude(group: 'org.spark-project.hive', module: 'hive-service') + exclude(group: 'org.spark-project.hive', module: 'hive-shims') + exclude(group: 'org.apache.thrift', module: 'libthrift') + exclude(group: 'org.slf4j', module: 'slf4j-api') + exclude(group: 'org.slf4j', module: 'slf4j-log4j12') + exclude(group: 'log4j', module: 'log4j') + exclude(group: 'commons-logging', module: 'commons-logging') + } + compile(group: 'org.spark-project.hive', name: 'hive-jdbc', version: hiveVersion) { + exclude(group: 'org.spark-project.hive', module: 'hive-common') + exclude(group: 'org.spark-project.hive', module: 'hive-metastore') + exclude(group: 'org.spark-project.hive', module: 'hive-serde') + exclude(group: 'org.spark-project.hive', module: 'hive-service') + exclude(group: 'org.spark-project.hive', module: 'hive-shims') + exclude(group: 'org.spark-project.hive', module: 'httpclient') + exclude(group: 'org.apache.curator', module: 'curator-framework') + exclude(group: 'org.apache.thrift', module: 'libthrift') + exclude(group: 'org.apache.thrift', module: 'libfb303') + exclude(group: 'org.apache.zookeeper', module: 'zookeeper') + exclude(group: 'org.slf4j', module: 'slf4j-api') + exclude(group: 'org.slf4j', module: 'slf4j-log4j12') + exclude(group: 'log4j', module: 'log4j') + exclude(group: 'commons-logging', module: 'commons-logging') + exclude(group: 'org.codehaus.groovy', module: 'groovy-all') + } + compile(group: 'net.sf.jpam', name: 'jpam', version: jpamVersion) { + exclude(group: 'javax.servlet', module: 'servlet-api') + } + + testCompile project(path: subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion, configuration: 'testOutput') + testCompile project(path: subprojectBase + 'snappy-spark-sql_' + scalaBinaryVersion, configuration: 'testOutput') + testCompile(group: 'org.seleniumhq.selenium', name: 'selenium-java', version: seleniumVersion) { + exclude(group: 'com.google.guava', module: 'guava') + exclude(group: 'io.netty', module: 'netty') + } + testCompile(group: 'org.seleniumhq.selenium', name: 'selenium-htmlunit-driver', version: seleniumVersion) { + exclude(group: 'com.google.guava', module: 'guava') + } +} + +// add generated sources +sourceSets.main.scala.srcDir 'src/gen/java' diff --git a/sql/hive/build.gradle b/sql/hive/build.gradle new file mode 100644 index 0000000000000..a3c22973264b2 --- /dev/null +++ b/sql/hive/build.gradle @@ -0,0 +1,117 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project Hive' + +dependencies { + compile project(subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-sql_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + + compile group: 'com.twitter', name: 'parquet-hadoop-bundle', version: hiveParquetVersion + compile group: 'org.apache.derby', name: 'derby', version: derbyVersion + compile(group: 'org.spark-project.hive', name: 'hive-exec', version: hiveVersion) { + exclude(group: 'org.spark-project.hive', module: 'hive-metastore') + exclude(group: 'org.spark-project.hive', module: 'hive-shims') + exclude(group: 'org.spark-project.hive', module: 'hive-ant') + exclude(group: 'org.spark-project.hive', module: 'spark-client') + exclude(group: 'org.apache.ant', module: 'ant') + exclude(group: 'com.esotericsoftware.kryo', module: 'kryo') + exclude(group: 'commons-codec', module: 'commons-codec') + exclude(group: 'commons-httpclient', module: 'commons-httpclient') + exclude(group: 'org.apache.avro', module: 'avro-mapred') + exclude(group: 'org.apache.calcite', module: 'calcite-core') + exclude(group: 'org.apache.curator', module: 'apache-curator') + exclude(group: 'org.apache.curator', module: 'curator-client') + exclude(group: 'org.apache.curator', module: 'curator-framework') + exclude(group: 'org.apache.thrift', module: 'libthrift') + exclude(group: 'org.apache.thrift', module: 'libfb303') + exclude(group: 'org.apache.zookeeper', module: 'zookeeper') + exclude(group: 'org.slf4j', module: 'slf4j-api') + exclude(group: 'org.slf4j', module: 'slf4j-log4j12') + exclude(group: 'log4j', module: 'log4j') + exclude(group: 'commons-logging', module: 'commons-logging') + exclude(group: 'org.codehaus.groovy', module: 'groovy-all') + } + compile(group: 'org.spark-project.hive', name: 'hive-metastore', version: hiveVersion) { + exclude(group: 'org.spark-project.hive', module: 'hive-serde') + exclude(group: 'org.spark-project.hive', module: 'hive-shims') + exclude(group: 'org.apache.thrift', module: 'libfb303') + exclude(group: 'org.apache.thrift', module: 'libthrift') + exclude(group: 'javax.servlet', module: 'servlet-api') + exclude(group: 'com.google.guava', module: 'guava') + exclude(group: 'org.slf4j', module: 'slf4j-api') + exclude(group: 'org.slf4j', module: 'slf4j-log4j12') + exclude(group: 'org.apache.derby', module: 'derby') + } + + compile group: 'org.apache.avro', name: 'avro', version: avroVersion + compile(group: 'org.apache.avro', name: 'avro-ipc', version: avroVersion) { + exclude(group: 'org.jboss.netty', module: 'netty') + exclude(group: 'org.mortbay.jetty', module: 'jetty') + exclude(group: 'org.mortbay.jetty', module: 'jetty-util') + exclude(group: 'org.mortbay.jetty', module: 'servlet-api') + exclude(group: 'org.apache.velocity', module: 'velocity') + } + compile(group: 'org.apache.avro', name: 'avro-mapred', version: avroVersion, classifier: 'hadoop2') { + exclude(group: 'org.jboss.netty', module: 'netty') + exclude(group: 'org.mortbay.jetty', module: 'jetty') + exclude(group: 'org.mortbay.jetty', module: 'jetty-util') + exclude(group: 'org.mortbay.jetty', module: 'servlet-api') + exclude(group: 'org.apache.velocity', module: 'velocity') + exclude(group: 'org.apache.avro', module: 'avro-ipc') + } + compile group: 'commons-httpclient', name: 'commons-httpclient', version: '3.1' + compile(group: 'org.apache.calcite', name: 'calcite-avatica', version: '1.2.0-incubating') { + exclude(group: 'com.fasterxml.jackson.core', module: 'jackson-annotations') + exclude(group: 'com.fasterxml.jackson.core', module: 'jackson-core') + exclude(group: 'com.fasterxml.jackson.core', module: 'jackson-databind') + } + compile(group: 'org.apache.calcite', name: 'calcite-core', version: '1.2.0-incubating') { + exclude(group: 'com.fasterxml.jackson.core', module: 'jackson-annotations') + exclude(group: 'com.fasterxml.jackson.core', module: 'jackson-core') + exclude(group: 'com.fasterxml.jackson.core', module: 'jackson-databind') + exclude(group: 'com.google.guava', module: 'guava') + exclude(group: 'com.google.code.findbugs', module: 'jsr305') + exclude(group: 'org.codehaus.janino', module: 'janino') + exclude(group: 'org.hsqldb', module: 'hsqldb') + exclude(group: 'org.pentaho', module: 'pentaho-aggdesigner-algorithm') + } + compile group: 'org.apache.httpcomponents', name: 'httpclient', version: httpClientVersion + compile group: 'org.codehaus.jackson', name: 'jackson-mapper-asl', version: '1.9.13' + compile group: 'commons-codec', name: 'commons-codec', version: commonsCodecVersion + compile group: 'joda-time', name: 'joda-time', version: '2.9.4' + compile group: 'org.jodd', name: 'jodd-core', version: '3.5.2' + compile group: 'com.google.code.findbugs', name: 'jsr305', version: jsr305Version + compile group: 'org.datanucleus', name: 'datanucleus-core', version: '3.2.10' + compile(group: 'org.apache.thrift', name: 'libthrift', version: thriftVersion) { + exclude(group: 'org.slf4j', module: 'slf4j-api') + } + compile(group: 'org.apache.thrift', name: 'libfb303', version: thriftVersion) { + exclude(group: 'org.slf4j', module: 'slf4j-api') + } + + testCompile group: 'org.apache.avro', name: 'avro-ipc', version: avroVersion, classifier: 'tests' + + testCompile project(path: subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion, configuration: 'testOutput') + testCompile project(path: subprojectBase + 'snappy-spark-sql_' + scalaBinaryVersion, configuration: 'testOutput') + testCompile project(path: subprojectBase + 'snappy-spark-catalyst_' + scalaBinaryVersion, configuration: 'testOutput') +} + +// fix scala+java test ordering +sourceSets.test.scala.srcDirs 'src/test/java', 'compatibility/src/test/scala' +sourceSets.test.java.srcDirs = [] diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 35f13a0626666..3af8af0814bb4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -14,24 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -/* - * Changes for SnappyData data platform. - * - * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. - * - * Licensed 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. See accompanying - * LICENSE file. - */ package org.apache.spark.sql.hive diff --git a/streaming/build.gradle b/streaming/build.gradle new file mode 100644 index 0000000000000..fcfb62199e447 --- /dev/null +++ b/streaming/build.gradle @@ -0,0 +1,44 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project Streaming' + +dependencies { + compile project(subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + + compile group: 'com.google.guava', name: 'guava', version: guavaVersion + compile group: 'org.eclipse.jetty', name: 'jetty-server', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-plus', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-util', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-http', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-servlet', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-servlets', version: jettyVersion + + testCompile project(path: subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion, configuration: 'testOutput') + testCompile(group: 'org.seleniumhq.selenium', name: 'selenium-java', version: seleniumVersion) { + exclude(group: 'com.google.guava', module: 'guava') + exclude(group: 'io.netty', module: 'netty') + } + testCompile(group: 'org.seleniumhq.selenium', name: 'selenium-htmlunit-driver', version: seleniumVersion) { + exclude(group: 'com.google.guava', module: 'guava') + } +} + +// fix scala+java mix to use scala first for tests +sourceSets.test.scala.srcDir "src/test/java" +sourceSets.test.java.srcDirs = [] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala index 41f7c60e1cc35..9618ec2507c76 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala @@ -14,6 +14,25 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + package org.apache.spark.streaming.rdd import java.nio.ByteBuffer diff --git a/tools/build.gradle b/tools/build.gradle new file mode 100644 index 0000000000000..05b48719a0d3e --- /dev/null +++ b/tools/build.gradle @@ -0,0 +1,25 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project Tools' + +dependencies { + compile group: 'org.scala-lang', name: 'scala-compiler', version: scalaVersion + compile group: 'org.clapper', name: 'classutil_' + scalaBinaryVersion, version: '1.0.12' +} + +// TODO: anything special required for deploy, install and source plugins in maven? diff --git a/yarn/build.gradle b/yarn/build.gradle new file mode 100644 index 0000000000000..f8eb23d2b5566 --- /dev/null +++ b/yarn/build.gradle @@ -0,0 +1,137 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Spark Project YARN' + +dependencies { + compile project(subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + + compile(group: 'org.apache.hadoop', name: 'hadoop-yarn-api', version: hadoopVersion) { + exclude(group: 'javax.servlet', module: 'servlet-api') + exclude(group: 'asm', module: 'asm') + exclude(group: 'org.ow2.asm', module: 'asm') + exclude(group: 'org.jboss.netty', module: 'netty') + exclude(group: 'commons-logging', module: 'commons-logging') + exclude(group: 'com.sun.jersey') + exclude(group: 'com.sun.jersey.jersey-test-framework') + exclude(group: 'com.sun.jersey.contribs') + } + compile(group: 'org.apache.hadoop', name: 'hadoop-yarn-common', version: hadoopVersion) { + exclude(group: 'asm', module: 'asm') + exclude(group: 'org.ow2.asm', module: 'asm') + exclude(group: 'org.jboss.netty', module: 'netty') + exclude(group: 'javax.servlet', module: 'servlet-api') + exclude(group: 'commons-logging', module: 'commons-logging') + exclude(group: 'com.sun.jersey') + exclude(group: 'com.sun.jersey.jersey-test-framework') + exclude(group: 'com.sun.jersey.contribs') + } + compile(group: 'org.apache.hadoop', name: 'hadoop-yarn-server-web-proxy', version: hadoopVersion) { + exclude(group: 'asm', module: 'asm') + exclude(group: 'org.ow2.asm', module: 'asm') + exclude(group: 'org.jboss.netty', module: 'netty') + exclude(group: 'javax.servlet', module: 'servlet-api') + exclude(group: 'commons-logging', module: 'commons-logging') + exclude(group: 'com.sun.jersey') + exclude(group: 'com.sun.jersey.jersey-test-framework') + exclude(group: 'com.sun.jersey.contribs') + } + compile(group: 'org.apache.hadoop', name: 'hadoop-yarn-client', version: hadoopVersion) { + exclude(group: 'asm', module: 'asm') + exclude(group: 'org.ow2.asm', module: 'asm') + exclude(group: 'org.jboss.netty', module: 'netty') + exclude(group: 'javax.servlet', module: 'servlet-api') + exclude(group: 'commons-logging', module: 'commons-logging') + exclude(group: 'com.sun.jersey') + exclude(group: 'com.sun.jersey.jersey-test-framework') + exclude(group: 'com.sun.jersey.contribs') + } + compile group: 'com.google.guava', name: 'guava', version: guavaVersion + compile group: 'org.eclipse.jetty', name: 'jetty-server', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-plus', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-util', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-http', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-servlet', version: jettyVersion + compile group: 'org.eclipse.jetty', name: 'jetty-servlets', version: jettyVersion + compile group: 'org.apache.derby', name: 'derby', version: derbyVersion + compile(group: 'org.spark-project.hive', name: 'hive-exec', version: hiveVersion) { + exclude(group: 'org.spark-project.hive', module: 'hive-metastore') + exclude(group: 'org.spark-project.hive', module: 'hive-shims') + exclude(group: 'org.spark-project.hive', module: 'hive-ant') + exclude(group: 'org.spark-project.hive', module: 'spark-client') + exclude(group: 'org.apache.ant', module: 'ant') + exclude(group: 'com.esotericsoftware.kryo', module: 'kryo') + exclude(group: 'commons-codec', module: 'commons-codec') + exclude(group: 'commons-httpclient', module: 'commons-httpclient') + exclude(group: 'org.apache.avro', module: 'avro-mapred') + exclude(group: 'org.apache.calcite', module: 'calcite-core') + exclude(group: 'org.apache.curator', module: 'apache-curator') + exclude(group: 'org.apache.curator', module: 'curator-client') + exclude(group: 'org.apache.curator', module: 'curator-framework') + exclude(group: 'org.apache.thrift', module: 'libthrift') + exclude(group: 'org.apache.thrift', module: 'libfb303') + exclude(group: 'org.apache.zookeeper', module: 'zookeeper') + exclude(group: 'org.slf4j', module: 'slf4j-api') + exclude(group: 'org.slf4j', module: 'slf4j-log4j12') + exclude(group: 'log4j', module: 'log4j') + exclude(group: 'commons-logging', module: 'commons-logging') + exclude(group: 'org.codehaus.groovy', module: 'groovy-all') + } + compile(group: 'org.spark-project.hive', name: 'hive-metastore', version: hiveVersion) { + exclude(group: 'org.spark-project.hive', module: 'hive-serde') + exclude(group: 'org.spark-project.hive', module: 'hive-shims') + exclude(group: 'org.apache.thrift', module: 'libfb303') + exclude(group: 'org.apache.thrift', module: 'libthrift') + exclude(group: 'javax.servlet', module: 'servlet-api') + exclude(group: 'com.google.guava', module: 'guava') + exclude(group: 'org.slf4j', module: 'slf4j-api') + exclude(group: 'org.slf4j', module: 'slf4j-log4j12') + exclude(group: 'org.apache.derby', module: 'derby') + } + compile(group: 'org.apache.thrift', name: 'libthrift', version: thriftVersion) { + exclude(group: 'org.slf4j', module: 'slf4j-api') + } + compile(group: 'org.apache.thrift', name: 'libfb303', version: thriftVersion) { + exclude(group: 'org.slf4j', module: 'slf4j-api') + } + + testCompile project(subprojectBase + 'snappy-spark-network-yarn_' + scalaBinaryVersion) + testCompile project(path: subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion, configuration: 'testOutput') + + testCompile group: 'org.eclipse.jetty.orbit', name: 'javax.servlet.jsp', version: '2.2.0.v201112011158' + testCompile group: 'org.eclipse.jetty.orbit', name: 'javax.servlet.jsp.jstl', version: '1.2.0.v201105211821' + testCompile(group: 'org.apache.hadoop', name: 'hadoop-yarn-server-tests', version: hadoopVersion, classifier:'tests') { + exclude(group: 'asm', module: 'asm') + exclude(group: 'org.ow2.asm', module: 'asm') + exclude(group: 'org.jboss.netty', module: 'netty') + exclude(group: 'javax.servlet', module: 'servlet-api') + exclude(group: 'commons-logging', module: 'commons-logging') + exclude(group: 'com.sun.jersey') + exclude(group: 'com.sun.jersey.jersey-test-framework') + exclude(group: 'com.sun.jersey.contribs') + } + testCompile(group: 'org.mortbay.jetty', name: 'jetty', version: '6.1.26') { + exclude(group: 'org.mortbay.jetty', module: 'servlet-api') + } + testCompile group: 'com.sun.jersey', name: 'jersey-core', version: sunJerseyVersion + testCompile group: 'com.sun.jersey', name: 'jersey-server', version: sunJerseyVersion + testCompile(group: 'com.sun.jersey', name: 'jersey-json', version: sunJerseyVersion) { + exclude(group: 'stax', module: 'stax-api') + } + testCompile group: 'com.sun.jersey.contribs', name: 'jersey-guice', version: sunJerseyVersion +} From ecb7f7ff72fb94dcad8ce6bda0a924d9f3d6d01b Mon Sep 17 00:00:00 2001 From: ymahajan Date: Mon, 30 Nov 2015 12:46:19 +0530 Subject: [PATCH 016/142] [SNAPPYDATA] Dynamic CQ changes in spark streaming - Allow registration of output streams on active StreamingContext - Added flag to DStream.initialize to allow initialization of newly added output streams with zeroTime - generatedRDDs made thread-safe --- .../spark/streaming/dstream/DStream.scala | 41 +++++++++++++++---- .../streaming/dstream/FileInputDStream.scala | 3 +- 2 files changed, 36 insertions(+), 8 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index e23edfa506517..479864f9d7051 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -19,8 +19,8 @@ package org.apache.spark.streaming.dstream import java.io.{IOException, ObjectInputStream, ObjectOutputStream} +import java.util.concurrent.ConcurrentHashMap -import scala.collection.mutable.HashMap import scala.language.implicitConversions import scala.reflect.ClassTag import scala.util.matching.Regex @@ -82,9 +82,17 @@ abstract class DStream[T: ClassTag] ( // Methods and fields available on all DStreams // ======================================================================= + import scala.collection.JavaConverters._ // RDDs generated, marked as private[streaming] so that testsuites can access it @transient - private[streaming] var generatedRDDs = new HashMap[Time, RDD[T]]() + // private[streaming] var generatedRDDs = new HashMap[Time, RDD[T]]() + private[streaming] var generatedRDDs: scala.collection.mutable.Map[Time, RDD[T]] = _ + + initGeneratedRDDs() + + def initGeneratedRDDs(): Unit = { + generatedRDDs = new ConcurrentHashMap[Time, RDD[T]]().asScala + } // Time zero for the DStream private[streaming] var zeroTime: Time = null @@ -184,12 +192,24 @@ abstract class DStream[T: ClassTag] ( this } + /** + * Initialize the DStream by setting the "zero" time, based on which + * the validity of future times is calculated. This method also recursively initializes + * its parent DStreams. + */ + private[streaming] def initialize(time: Time) { + initialize(time, skipInitialized = false) + } + /** * Initialize the DStream by setting the "zero" time, based on which * the validity of future times is calculated. This method also recursively initializes * its parent DStreams. */ - private[streaming] def initialize(time: Time) { + private[streaming] def initialize(time: Time, skipInitialized: Boolean) { + if (skipInitialized && isInitialized) { + return + } if (zeroTime != null && zeroTime != time) { throw new SparkException(s"ZeroTime is already initialized to $zeroTime" + s", cannot initialize it again to $time") @@ -213,7 +233,7 @@ abstract class DStream[T: ClassTag] ( } // Initialize the dependencies - dependencies.foreach(_.initialize(zeroTime)) + dependencies.foreach(_.initialize(zeroTime, skipInitialized)) } private def validateAtInit(): Unit = { @@ -221,9 +241,11 @@ abstract class DStream[T: ClassTag] ( case StreamingContextState.INITIALIZED => // good to go case StreamingContextState.ACTIVE => + /* throw new IllegalStateException( "Adding new inputs, transformations, and output operations after " + "starting a context is not supported") + */ case StreamingContextState.STOPPED => throw new IllegalStateException( "Adding new inputs, transformations, and output operations after " + @@ -535,7 +557,8 @@ abstract class DStream[T: ClassTag] ( private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { logDebug(s"${this.getClass().getSimpleName}.readObject used") ois.defaultReadObject() - generatedRDDs = new HashMap[Time, RDD[T]]() + // generatedRDDs = new HashMap[Time, RDD[T]]() + initGeneratedRDDs() } // ======================================================================= @@ -651,8 +674,12 @@ abstract class DStream[T: ClassTag] ( private def foreachRDD( foreachFunc: (RDD[T], Time) => Unit, displayInnerRDDOps: Boolean): Unit = { - new ForEachDStream(this, - context.sparkContext.clean(foreachFunc, false), displayInnerRDDOps).register() + val dStream = new ForEachDStream(this, + context.sparkContext.clean(foreachFunc, false), displayInnerRDDOps) + if (ssc.getState() == StreamingContextState.ACTIVE) { + dStream.initialize(ssc.graph.zeroTime, skipInitialized = true) + } + dStream.register() } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index b8a5a96faf15c..ae97ba0a5d823 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -316,7 +316,8 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { logDebug(this.getClass().getSimpleName + ".readObject used") ois.defaultReadObject() - generatedRDDs = new mutable.HashMap[Time, RDD[(K, V)]]() + // generatedRDDs = new mutable.HashMap[Time, RDD[(K, V)]]() + initGeneratedRDDs() batchTimeToSelectedFiles = new mutable.HashMap[Time, Array[String]] recentlySelectedFiles = new mutable.HashSet[String]() fileToModTime = new TimeStampedHashMap[String, Long](true) From 40a4ea0dea8a5013a44cb2e299044fb4d9f4d676 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Mon, 8 Aug 2016 14:47:58 +0530 Subject: [PATCH 017/142] [SNAPPYDATA] Fix cluster startup due to executionId format Allow for non-integral values as executionId (SnappyData uses full DistributedMember representation) --- .../cluster/CoarseGrainedSchedulerBackend.scala | 10 ++++++++-- .../scala/org/apache/spark/sql/AnalysisException.scala | 5 ----- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 4d75063fbf1c5..74a19b91b253b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -195,8 +195,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // in this block are read when requesting executors CoarseGrainedSchedulerBackend.this.synchronized { executorDataMap.put(executorId, data) - if (currentExecutorIdCounter < executorId.toInt) { - currentExecutorIdCounter = executorId.toInt + // [snappydata] skip toInt used for Yarn since snappydata's + // executorId is not an integer + try { + if (currentExecutorIdCounter < executorId.toInt) { + currentExecutorIdCounter = executorId.toInt + } + } catch { + case nfe: NumberFormatException => // ignore } if (numPendingExecutors > 0) { numPendingExecutors -= 1 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index 2b2aa60f2538c..741d2aaa3061d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -54,11 +54,6 @@ class AnalysisException protected[sql] ( val cause: Option[Throwable] = None) extends Exception(message, cause.orNull) with Serializable { - def this(message: String, cause: Throwable) = { - this(message) - initCause(cause) - } - def withPosition(line: Option[Int], startPosition: Option[Int]): AnalysisException = { val newException = new AnalysisException(message, line, startPosition) newException.setStackTrace(getStackTrace) From 37ee5141e309a907cde2d4808bc5d86a8ffbb2b4 Mon Sep 17 00:00:00 2001 From: nthanvi Date: Wed, 2 Dec 2015 17:05:06 +0530 Subject: [PATCH 018/142] [SNAPPYDATA] Accept Spark properties with "snappydata." prefix - in addition to properties with "spark." prefix, also accept "snappydata." prefix in system properties, spark submit/shell Conflicts: launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java --- .../scala/org/apache/spark/SparkConf.scala | 21 ++++++++++++++- .../scala/org/apache/spark/SparkEnv.scala | 21 ++++++++++++++- .../spark/deploy/SparkSubmitArguments.scala | 20 +++++++++++++- .../scala/org/apache/spark/util/Utils.scala | 2 +- .../apache/spark/launcher/SparkLauncher.java | 26 +++++++++++++++++++ 5 files changed, 86 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index f53b2bed74c6e..d6ece44c7173f 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark @@ -72,7 +90,8 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria private[spark] def loadFromSystemProperties(silent: Boolean): SparkConf = { // Load any spark.* system properties - for ((key, value) <- Utils.getSystemProperties if key.startsWith("spark.")) { + for ((key, value) <- Utils.getSystemProperties + if key.startsWith("spark.") || key.startsWith("snappydata.")) { set(key, value, silent) } this diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 72123f2232532..2ff379cced1a8 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark @@ -431,7 +449,8 @@ object SparkEnv extends Logging { // System properties that are not java classpaths val systemProperties = Utils.getSystemProperties.toSeq val otherProperties = systemProperties.filter { case (k, _) => - k != "java.class.path" && !k.startsWith("spark.") + k != "java.class.path" && !k.startsWith("spark.") && + !k.startsWith("snappydata.") }.sorted // Class paths including all added jars and files diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 9db7a1fe3106d..c962b4df84e80 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.deploy @@ -138,7 +156,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S */ private def ignoreNonSparkProperties(): Unit = { sparkProperties.foreach { case (k, v) => - if (!k.startsWith("spark.")) { + if (!k.startsWith("spark.") && !k.startsWith("snappydata.")) { sparkProperties -= k SparkSubmit.printWarning(s"Ignoring non-spark config property: $k=$v") } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index be5becf145d0f..5706a7ba35ff3 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2105,7 +2105,7 @@ private[spark] object Utils extends Logging { val path = Option(filePath).getOrElse(getDefaultPropertiesFile()) Option(path).foreach { confFile => getPropertiesFromFile(confFile).filter { case (k, v) => - k.startsWith("spark.") + k.startsWith("spark.") || k.startsWith("snappydata.") }.foreach { case (k, v) => conf.setIfMissing(k, v) sys.props.getOrElseUpdate(k, v) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index f86d40015bd22..fd6763e13ed35 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.launcher; @@ -170,6 +188,14 @@ public SparkLauncher directory(File dir) { */ public SparkLauncher redirectError() { redirectErrorStream = true; + } + + public SparkLauncher setConf(String key, String value) { + checkNotNull(key, "key"); + checkNotNull(value, "value"); + checkArgument(key.startsWith("spark.") || key.startsWith("snappydata."), + "'key' must start with 'spark.' or 'snappydata.'"); + builder.conf.put(key, value); return this; } From 53f2d15fd6f5d640c8d881dab4de0fc8bba792db Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Tue, 16 Aug 2016 13:07:56 +0530 Subject: [PATCH 019/142] [SNAPPYDATA] More fixes for SnappyData for Spark 2.0 - make AbstractDataType.classTag as lazy to optimize DataType object creation (e.g. DecimalType) - allow for more than 16 bytes in serialized Decimal objects (precision has been increased to 127 in SnappyData) - minor updates to build.gradle's including a proper dependency for generateBuildInfo to avoid its re-run (and thus all dependent projects) every time - add modification headers for touched files (and remove from a couple of old ones which have later been reverted) - updating dependencies as per latest merge from branch-2.0 - fix scalastyle errors Conflicts: sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala Conflicts: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala --- .gitignore | 2 + assembly/build.gradle | 2 +- build.gradle | 11 ++- common/network-yarn/build.gradle | 16 ++-- core/build.gradle | 19 ++-- .../CoarseGrainedSchedulerBackend.scala | 18 ++++ external/flume-sink/build.gradle | 4 +- external/kafka-0-10/build.gradle | 2 +- repl/build.gradle | 4 +- settings.gradle | 2 +- .../codegen/UnsafeArrayWriter.java | 20 +++- .../apache/spark/sql/AnalysisException.scala | 18 ---- .../sql/catalyst/CatalystTypeConverters.scala | 1 + .../codegen/GenerateUnsafeProjection.scala | 28 +----- .../plans/physical/partitioning.scala | 91 +++++++++++++++++++ .../spark/sql/types/AbstractDataType.scala | 23 +++++ .../apache/spark/sql/types/DecimalType.scala | 3 +- .../parquet/ParquetSchemaConverter.scala | 3 +- sql/hive/build.gradle | 1 + streaming/build.gradle | 2 +- .../spark/streaming/dstream/DStream.scala | 26 +++++- .../streaming/dstream/FileInputDStream.scala | 18 ++++ yarn/build.gradle | 1 + 23 files changed, 239 insertions(+), 76 deletions(-) diff --git a/.gitignore b/.gitignore index 6e8668c46a350..ae8da5c979079 100644 --- a/.gitignore +++ b/.gitignore @@ -86,8 +86,10 @@ spark-warehouse/ # For R session data .RData .RHistory +.Rhistory *.Rproj *.Rproj.* + .Rproj.user # gradle specific diff --git a/assembly/build.gradle b/assembly/build.gradle index 5a0f18c45e3f6..63db32e3e41f6 100644 --- a/assembly/build.gradle +++ b/assembly/build.gradle @@ -113,7 +113,7 @@ task product(type: Zip) { into snappyProductDir } def releaseFile = file("${snappyProductDir}/RELEASE") - String buildFlags = "" + String buildFlags = '' if (rootProject.hasProperty('docker')) { buildFlags += ' -Pdocker' } diff --git a/build.gradle b/build.gradle index c235dcf712cc4..dddce0ab57942 100644 --- a/build.gradle +++ b/build.gradle @@ -80,6 +80,7 @@ allprojects { commonsCodecVersion = '1.10' avroVersion = '1.7.7' jsr305Version = '3.0.1' + jlineVersion = '2.14.2' scalatestVersion = '2.2.6' pegdownVersion = '1.6.0' @@ -326,19 +327,19 @@ subprojects { def eol = System.getProperty('line.separator') beforeTest { desc -> def now = new Date().format('yyyy-MM-dd HH:mm:ss.SSS Z') - def progress = new File(workingDir, "progress.txt") - def output = new File(workingDir, "output.txt") + def progress = new File(workingDir, 'progress.txt') + def output = new File(workingDir, 'output.txt') progress << "$now Starting test $desc.className $desc.name$eol" output << "${now} STARTING TEST ${desc.className} ${desc.name}${eol}${eol}" } onOutput { desc, event -> - def output = new File(workingDir, "output.txt") + def output = new File(workingDir, 'output.txt') output << event.message } afterTest { desc, result -> def now = new Date().format('yyyy-MM-dd HH:mm:ss.SSS Z') - def progress = new File(workingDir, "progress.txt") - def output = new File(workingDir, "output.txt") + def progress = new File(workingDir, 'progress.txt') + def output = new File(workingDir, 'output.txt') progress << "${now} Completed test ${desc.className} ${desc.name} with result: ${result.resultType}${eol}" output << "${eol}${now} COMPLETED TEST ${desc.className} ${desc.name} with result: ${result.resultType}${eol}${eol}" result.exceptions.each { t -> diff --git a/common/network-yarn/build.gradle b/common/network-yarn/build.gradle index 323463acad43c..bbb6d8c7f81ae 100644 --- a/common/network-yarn/build.gradle +++ b/common/network-yarn/build.gradle @@ -75,19 +75,19 @@ shadowJar { relocate 'com.fasterxml.jackson', "${shadePackageName}.com.fasterxml.jackson" relocate 'com.google.common', "${shadePackageName}.guava" - String createdBy = "" + String createdBy = '' if (rootProject.hasProperty('enablePublish')) { - createdBy = "SnappyData Build Team" + createdBy = 'SnappyData Build Team' } else { - createdBy = System.getProperty("user.name") + createdBy = System.getProperty('user.name') } manifest { attributes( - "Manifest-Version" : "1.0", - "Created-By" : createdBy, - "Title" : project.name, - "Version" : version, - "Vendor" : "SnappyData, Inc." + 'Manifest-Version' : '1.0', + 'Created-By' : createdBy, + 'Title' : project.name, + 'Version' : version, + 'Vendor' : 'SnappyData, Inc.' ) } diff --git a/core/build.gradle b/core/build.gradle index e1b52bfec0799..9395a129dac33 100644 --- a/core/build.gradle +++ b/core/build.gradle @@ -57,6 +57,7 @@ dependencies { exclude(group: 'org.ow2.asm', module: 'asm') exclude(group: 'org.apache.zookeeper', module: 'zookeeper') exclude(group: 'org.jboss.netty', module: 'netty') + exclude(group: 'jline', module: 'jline') exclude(group: 'commons-logging', module: 'commons-logging') exclude(group: 'org.mockito', module: 'mockito-all') exclude(group: 'org.mortbay.jetty', module: 'servlet-api-2.5') @@ -142,6 +143,7 @@ dependencies { testCompile(group: 'org.apache.curator', name: 'curator-test', version: curatorVersion) { exclude(group: 'org.apache.zookeeper', module: 'zookeeper') exclude(group: 'org.jboss.netty', module: 'netty') + exclude(group: 'jline', module: 'jline') exclude(group: 'com.google.guava', module: 'guava') } } @@ -155,12 +157,17 @@ sourceSets.test.java.srcDirs = [] // generate properties using spark-build-info and add to project resources String extraResourceDir = "${buildDir}/extra-resources" -task generateBuildInfo << { - file(extraResourceDir).mkdirs() - exec { - executable 'bash' - workingDir = buildDir - args "${projectDir}/../build/spark-build-info", extraResourceDir, version +task generateBuildInfo { + outputs.file "${extraResourceDir}/spark-version-info.properties" + inputs.dir compileScala.destinationDir + + doLast { + file(extraResourceDir).mkdirs() + exec { + executable 'bash' + workingDir = buildDir + args "${projectDir}/../build/spark-build-info", extraResourceDir, version + } } } sourceSets { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 74a19b91b253b..7ffae9b68cb0e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.scheduler.cluster diff --git a/external/flume-sink/build.gradle b/external/flume-sink/build.gradle index 99a0357048896..3a0060cca4897 100644 --- a/external/flume-sink/build.gradle +++ b/external/flume-sink/build.gradle @@ -41,9 +41,9 @@ dependencies { compile group: 'io.netty', name: 'netty', version: nettyVersion } -// for compatibility with maven generated code, though default "string" seems +// for compatibility with maven generated code, though default 'string' seems // more efficient requiring no conversions -avro.stringType = "charSequence" +avro.stringType = 'charSequence' tasks.withType(JavaCompile) { options.compilerArgs << '-Xlint:all,-serial,-path,-deprecation,-unchecked' diff --git a/external/kafka-0-10/build.gradle b/external/kafka-0-10/build.gradle index c898dfce495c5..d5dc6611a97f0 100644 --- a/external/kafka-0-10/build.gradle +++ b/external/kafka-0-10/build.gradle @@ -21,7 +21,7 @@ dependencies { compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) compile project(subprojectBase + 'snappy-spark-streaming_' + scalaBinaryVersion) - compile(group: 'org.apache.kafka', name: 'kafka_' + scalaBinaryVersion, version: '0.10.0.0') { + compile(group: 'org.apache.kafka', name: 'kafka_' + scalaBinaryVersion, version: '0.10.0.1') { exclude(group: 'com.sun.jmx', module: 'jmxri') exclude(group: 'com.sun.jdmk ', module: 'jmxtools') exclude(group: 'net.sf.jopt-simple', module: 'jopt-simple') diff --git a/repl/build.gradle b/repl/build.gradle index 1e51600f39b04..018602803e4f2 100644 --- a/repl/build.gradle +++ b/repl/build.gradle @@ -25,9 +25,7 @@ dependencies { compile group: 'org.apache.xbean', name: 'xbean-asm5-shaded', version: '4.4' compile group: 'org.scala-lang', name: 'scala-compiler', version: scalaVersion compile group: 'org.slf4j', name: 'jul-to-slf4j', version: slf4jVersion - if (scalaBinaryVersion == '2.10') { - compile group: 'org.scala-lang', name: 'jline', version:scalaVersion - } + compile group: 'jline', name: 'jline', version: jlineVersion runtime project(subprojectBase + 'snappy-spark-mllib_' + scalaBinaryVersion) diff --git a/settings.gradle b/settings.gradle index 70ea595ee61a5..ca33d18d94bf5 100644 --- a/settings.gradle +++ b/settings.gradle @@ -15,7 +15,7 @@ * LICENSE file. */ -def scalaBinaryVersion = "2.11" +def scalaBinaryVersion = '2.11' rootProject.name = 'snappy-spark' include ':snappy-spark-tags_' + scalaBinaryVersion diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index 791e8d80e6cba..c58e90cad50d9 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.catalyst.expressions.codegen; @@ -190,7 +208,7 @@ public void write(int ordinal, Decimal input, int precision, int scale) { } else { final byte[] bytes = input.toJavaBigDecimal().unscaledValue().toByteArray(); final int numBytes = bytes.length; - assert numBytes <= 16; + // assert numBytes <= 16; int roundedSize = ByteArrayMethods.roundNumberOfBytesToNearestWord(numBytes); holder.grow(roundedSize); diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index 741d2aaa3061d..50ee6cd4085ea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -14,24 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -/* - * Changes for SnappyData data platform. - * - * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. - * - * Licensed 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. See accompanying - * LICENSE file. - */ package org.apache.spark.sql diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index 86a625a6c8272..c40d712817791 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -188,6 +188,7 @@ object CatalystTypeConverters { convertedIterable += elementConverter.toCatalyst(item) } new GenericArrayData(convertedIterable.toArray) + case a: ArrayData => a } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index d9e9c866aeb5b..b8ccf6e8e4ad7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -14,24 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -/* - * Changes for SnappyData data platform. - * - * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. - * - * Licensed 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. See accompanying - * LICENSE file. - */ package org.apache.spark.sql.catalyst.expressions.codegen @@ -62,7 +44,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro } // TODO: if the nullability of field is correct, we can use it to save null check. - private[sql] def writeStructToBuffer( + private def writeStructToBuffer( ctx: CodegenContext, input: String, fieldTypes: Seq[DataType], @@ -195,7 +177,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro } // TODO: if the nullability of array element is correct, we can use it to save null check. - private[sql] def writeArrayToBuffer( + private def writeArrayToBuffer( ctx: CodegenContext, input: String, elementType: DataType, @@ -231,13 +213,13 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro $arrayWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); """ - case a @ ArrayType(at, _) => + case a @ ArrayType(et, _) => s""" final int $tmpCursor = $bufferHolder.cursor; ${writeArrayToBuffer(ctx, element, et, bufferHolder)} $arrayWriter.setOffsetAndSize($index, $tmpCursor, $bufferHolder.cursor - $tmpCursor); $arrayWriter.setOffset($index); - ${writeArrayToBuffer(ctx, element, at, bufferHolder)} + ${writeArrayToBuffer(ctx, element, et, bufferHolder)} """ case m @ MapType(kt, vt, _) => @@ -276,7 +258,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro } // TODO: if the nullability of value element is correct, we can use it to save null check. - private[sql] def writeMapToBuffer( + private def writeMapToBuffer( ctx: CodegenContext, input: String, keyType: DataType, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 77b40e6452880..e488281de0dc1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -209,6 +209,97 @@ case object SinglePartition extends Partitioning { case ClusteredDistribution(_, Some(requiredNumPartitions)) => requiredNumPartitions == 1 case _ => true } + + override def compatibleWith(other: Partitioning): Boolean = other.numPartitions == 1 + + override def guarantees(other: Partitioning): Boolean = other.numPartitions == 1 +} + +/** + * Represents a partitioning where rows are split up across partitions based on the hash + * of `expressions`. All rows where `expressions` evaluate to the same values are guaranteed to be + * in the same partition. Moreover while evaluating expressions if they are given in different order + * than this partitioning then also it is considered equal. + */ +case class OrderlessHashPartitioning(expressions: Seq[Expression], numPartitions: Int) + extends Expression with Partitioning with Unevaluable { + + + override def children: Seq[Expression] = expressions + override def nullable: Boolean = false + override def dataType: DataType = IntegerType + + private def matchExpressions(otherExpression: Seq[Expression]): Boolean = { + expressions.length == otherExpression.length && expressions.forall(a => + otherExpression.exists(e => e.semanticEquals(a))) + } + + override def satisfies(required: Distribution): Boolean = required match { + case UnspecifiedDistribution => true + case ClusteredDistribution(requiredClustering) => { + matchExpressions(requiredClustering) + } + case _ => false + } + + private def anyOrderEquals(other: HashPartitioning) : Boolean = { + other.numPartitions == this.numPartitions && + matchExpressions(other.expressions) + } + + override def compatibleWith(other: Partitioning): Boolean = other match { + case p: HashPartitioning => anyOrderEquals(p) + case _ => false + } + + override def guarantees(other: Partitioning): Boolean = other match { + case o: HashPartitioning => anyOrderEquals(o) + case _ => false + } + +} + +/** + * Represents a partitioning where rows are split up across partitions based on the hash + * of `expressions`. All rows where `expressions` evaluate to the same values are guaranteed to be + * in the same partition. Moreover while evaluating expressions if they are given in different order + * than this partitioning then also it is considered equal. + */ +case class OrderlessHashPartitioning(expressions: Seq[Expression], numPartitions: Int) + extends Expression with Partitioning with Unevaluable { + + + override def children: Seq[Expression] = expressions + override def nullable: Boolean = false + override def dataType: DataType = IntegerType + + private def matchExpressions(otherExpression: Seq[Expression]): Boolean = { + expressions.length == otherExpression.length && expressions.forall(a => + otherExpression.exists(e => e.semanticEquals(a))) + } + + override def satisfies(required: Distribution): Boolean = required match { + case UnspecifiedDistribution => true + case ClusteredDistribution(requiredClustering) => + matchExpressions(requiredClustering) + case _ => false + } + + private def anyOrderEquals(other: HashPartitioning) : Boolean = { + other.numPartitions == this.numPartitions && + matchExpressions(other.expressions) + } + + override def compatibleWith(other: Partitioning): Boolean = other match { + case p: HashPartitioning => anyOrderEquals(p) + case _ => false + } + + override def guarantees(other: Partitioning): Boolean = other match { + case o: HashPartitioning => anyOrderEquals(o) + case _ => false + } + } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala index 3041f44b116ea..f405f1ea2e128 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.sql.types @@ -115,6 +133,11 @@ protected[sql] abstract class AtomicType extends DataType { private[sql] type InternalType private[sql] val tag: TypeTag[InternalType] private[sql] val ordering: Ordering[InternalType] + + @transient private[sql] lazy val classTag = ScalaReflectionLock.synchronized { + val mirror = runtimeMirror(Utils.getSparkClassLoader) + ClassTag[InternalType](mirror.runtimeClass(tag.tpe)) + } } object AtomicType { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala index c19067c73955f..117158aa500a2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala @@ -66,7 +66,8 @@ case class DecimalType(precision: Int, scale: Int) extends FractionalType { } if (precision > DecimalType.MAX_PRECISION) { - throw new AnalysisException(s"DecimalType can only support precision up to ${DecimalType.MAX_PRECISION}") + throw new AnalysisException( + s"DecimalType can only support precision up to ${DecimalType.MAX_PRECISION}") } // default constructor for Java diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala index 6eeaa02388379..530ee180218a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala @@ -612,7 +612,8 @@ private[sql] object ParquetSchemaConverter { } // Returns the minimum number of bytes needed to store a decimal with a given `precision`. - val minBytesForPrecision = Array.tabulate[Int](DecimalType.MAX_PRECISION + 1)(computeMinBytesForPrecision) + val minBytesForPrecision = Array.tabulate[Int](DecimalType.MAX_PRECISION + 1)( + computeMinBytesForPrecision) // Max precision of a decimal value stored in `numBytes` bytes def maxPrecisionForBytes(numBytes: Int): Int = { diff --git a/sql/hive/build.gradle b/sql/hive/build.gradle index a3c22973264b2..25f6d76d4d7eb 100644 --- a/sql/hive/build.gradle +++ b/sql/hive/build.gradle @@ -46,6 +46,7 @@ dependencies { exclude(group: 'log4j', module: 'log4j') exclude(group: 'commons-logging', module: 'commons-logging') exclude(group: 'org.codehaus.groovy', module: 'groovy-all') + exclude(group: 'jline', module: 'jline') } compile(group: 'org.spark-project.hive', name: 'hive-metastore', version: hiveVersion) { exclude(group: 'org.spark-project.hive', module: 'hive-serde') diff --git a/streaming/build.gradle b/streaming/build.gradle index fcfb62199e447..bcfeecb82f8ff 100644 --- a/streaming/build.gradle +++ b/streaming/build.gradle @@ -40,5 +40,5 @@ dependencies { } // fix scala+java mix to use scala first for tests -sourceSets.test.scala.srcDir "src/test/java" +sourceSets.test.scala.srcDir 'src/test/java' sourceSets.test.java.srcDirs = [] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 479864f9d7051..039c401ace9af 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.streaming.dstream @@ -193,10 +211,10 @@ abstract class DStream[T: ClassTag] ( } /** - * Initialize the DStream by setting the "zero" time, based on which - * the validity of future times is calculated. This method also recursively initializes - * its parent DStreams. - */ + * Initialize the DStream by setting the "zero" time, based on which + * the validity of future times is calculated. This method also recursively initializes + * its parent DStreams. + */ private[streaming] def initialize(time: Time) { initialize(time, skipInitialized = false) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index ae97ba0a5d823..cb8a2443bf1f1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -14,6 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/* + * Changes for SnappyData data platform. + * + * Portions Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ package org.apache.spark.streaming.dstream diff --git a/yarn/build.gradle b/yarn/build.gradle index f8eb23d2b5566..5fb389cba38c0 100644 --- a/yarn/build.gradle +++ b/yarn/build.gradle @@ -91,6 +91,7 @@ dependencies { exclude(group: 'log4j', module: 'log4j') exclude(group: 'commons-logging', module: 'commons-logging') exclude(group: 'org.codehaus.groovy', module: 'groovy-all') + exclude(group: 'jline', module: 'jline') } compile(group: 'org.spark-project.hive', name: 'hive-metastore', version: hiveVersion) { exclude(group: 'org.spark-project.hive', module: 'hive-serde') From 3e82acba88ab28c493392ab8be5cb709b5b69739 Mon Sep 17 00:00:00 2001 From: nthanvi Date: Thu, 25 Aug 2016 12:39:02 +0530 Subject: [PATCH 020/142] Snap 293 (#1) * Using the scala collections compiler was not able to differentiate between the scala and java api * Minor changes for snappy implementation of executor This is required as we need to have a classloader that also looks into the snappy store for the classes. * Revert "Using the scala collections" This reverts commit c2ab0c5aa3974337277560fb1c8a9d0c3661ec09. Conflicts: core/src/main/scala/org/apache/spark/executor/Executor.scala --- .../spark/executor/CoarseGrainedExecutorBackend.scala | 5 ++++- .../src/main/scala/org/apache/spark/executor/Executor.scala | 6 +++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 8cc65f6e4efa7..fab79c8904c07 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -71,6 +71,9 @@ private[spark] class CoarseGrainedExecutorBackend( }(ThreadUtils.sameThread) } + protected def registerExecutor: Executor = + new Executor(executorId, hostname, env, userClassPath, isLocal = false) + def extractLogUrls: Map[String, String] = { val prefix = "SPARK_LOG_URL_" sys.env.filterKeys(_.startsWith(prefix)) @@ -81,7 +84,7 @@ private[spark] class CoarseGrainedExecutorBackend( case RegisteredExecutor => logInfo("Successfully registered with driver") try { - executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false) + executor = registerExecutor } catch { case NonFatal(e) => exitExecutor(1, "Unable to create executor due to " + e.getMessage, e) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 05ec6565dfe25..b3b107e5a49fd 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -37,7 +37,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.memory.{SparkOutOfMemoryError, TaskMemoryManager} import org.apache.spark.rpc.RpcTimeout -import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task, TaskDescription} +import org.apache.spark.scheduler.{AccumulableInfo, DirectTaskResult, IndirectTaskResult, Task, TaskDescription} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} import org.apache.spark.util._ @@ -64,7 +64,7 @@ private[spark] class Executor( // Application dependencies (added through SparkContext) that we've fetched so far on this node. // Each map holds the master's timestamp for the version of that file or JAR we got. private val currentFiles: HashMap[String, Long] = new HashMap[String, Long]() - private val currentJars: HashMap[String, Long] = new HashMap[String, Long]() + protected val currentJars: HashMap[String, Long] = new HashMap[String, Long]() private val EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new Array[Byte](0)) @@ -697,7 +697,7 @@ private[spark] class Executor( * Create a ClassLoader for use in tasks, adding any JARs specified by the user or any classes * created by the interpreter to the search path */ - private def createClassLoader(): MutableURLClassLoader = { + protected def createClassLoader(): MutableURLClassLoader = { // Bootstrap the list of jars with the user class path. val now = System.currentTimeMillis() userClassPath.foreach { url => From 27025d1bd767caf70f6bd480fdf4c24409dd8dea Mon Sep 17 00:00:00 2001 From: ymahajan Date: Fri, 2 Sep 2016 16:53:33 +0530 Subject: [PATCH 021/142] [SNAPPYDATA] fix a scalaStyle issue Conflicts: core/src/main/scala/org/apache/spark/Partitioner.scala sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala --- core/src/main/scala/org/apache/spark/Partitioner.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index c940cb25d478b..bdfecd08f27c8 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -109,6 +109,12 @@ object Partitioner { class HashPartitioner(partitions: Int) extends Partitioner { require(partitions >= 0, s"Number of partitions ($partitions) cannot be negative.") + require(buckets >= 0, s"Number of buckets ($buckets) cannot be negative.") + + def this(partitions: Int) { + this(partitions, 0) + } + def numPartitions: Int = partitions def getPartition(key: Any): Int = key match { From 9ac66618630b6f0456c4cb0b83a67966a9a30478 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Wed, 7 Sep 2016 18:46:12 +0530 Subject: [PATCH 022/142] [SNAP-966] Prefer conversions to date/timestamp and not strings (#7) - for all cases of implicit casts, convert to date or timestamp values instead of string when one side is a string - likewise when one side is a timestamp and other date then both are being converted to string; now convert date to timestamp Conflicts: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala --- .../sql/catalyst/analysis/TypeCoercion.scala | 37 +++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index e8669c4637d06..8921683a0e033 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -345,6 +345,43 @@ object TypeCoercion { if findCommonTypeForBinaryComparison(left.dataType, right.dataType).isDefined => val commonType = findCommonTypeForBinaryComparison(left.dataType, right.dataType).get p.makeCopy(Array(castExpr(left, commonType), castExpr(right, commonType))) + // We should cast all relative timestamp/date/string comparison into string comparisons + // This behaves as a user would expect because timestamp strings sort lexicographically. + // i.e. TimeStamp(2013-01-01 00:00 ...) < "2014" = true + case p @ BinaryComparison(left @ StringType(), right @ DateType()) => + p.makeCopy(Array(left, Cast(right, StringType))) + case p @ BinaryComparison(left @ DateType(), right @ StringType()) => + p.makeCopy(Array(Cast(left, StringType), right)) + case p @ BinaryComparison(left @ StringType(), right @ TimestampType()) => + p.makeCopy(Array(left, Cast(right, StringType))) + case p @ BinaryComparison(left @ TimestampType(), right @ StringType()) => + p.makeCopy(Array(Cast(left, StringType), right)) + + // Comparisons between dates and timestamps. + case p @ BinaryComparison(left @ TimestampType(), right @ DateType()) => + p.makeCopy(Array(Cast(left, StringType), Cast(right, StringType))) + case p @ BinaryComparison(left @ DateType(), right @ TimestampType()) => + p.makeCopy(Array(Cast(left, StringType), Cast(right, StringType))) + + // Checking NullType + case p @ BinaryComparison(left @ StringType(), right @ NullType()) => + p.makeCopy(Array(left, Literal.create(null, StringType))) + case p @ BinaryComparison(left @ NullType(), right @ StringType()) => + p.makeCopy(Array(Literal.create(null, StringType), right)) + + case p @ BinaryComparison(left @ StringType(), right) if right.dataType != StringType => + p.makeCopy(Array(Cast(left, DoubleType), right)) + case p @ BinaryComparison(left, right @ StringType()) if left.dataType != StringType => + p.makeCopy(Array(left, Cast(right, DoubleType))) + + case i @ In(a @ DateType(), b) if b.forall(_.dataType == StringType) => + i.makeCopy(Array(Cast(a, StringType), b)) + case i @ In(a @ TimestampType(), b) if b.forall(_.dataType == StringType) => + i.makeCopy(Array(a, b.map(Cast(_, TimestampType)))) + case i @ In(a @ DateType(), b) if b.forall(_.dataType == TimestampType) => + i.makeCopy(Array(Cast(a, StringType), b.map(Cast(_, StringType)))) + case i @ In(a @ TimestampType(), b) if b.forall(_.dataType == DateType) => + i.makeCopy(Array(Cast(a, StringType), b.map(Cast(_, StringType)))) case Abs(e @ StringType()) => Abs(Cast(e, DoubleType)) case Sum(e @ StringType()) => Sum(Cast(e, DoubleType)) From 8455f59f3d57b219e30b23384d57ceb95f919684 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Wed, 7 Sep 2016 21:11:26 +0530 Subject: [PATCH 023/142] [SNAP-1034] Optimizations at Spark layer as seen in profiling (#10) - added a aggBufferAttributeForGroup to aggregates to be used to avoid nullable checks in generated code in aggregate buffers used in HashAggregateExec (if aggregate is on zero rows, then there will be no row in the map); accompanying "initialValuesForGroup" added for initial aggregation buffer values - use OpenHashMap in DictionaryEncoding which is faster than normal hash map; added clear methods to OpenHashMap/OpenHashSet for reuse - minor correction in the string in HiveUtils Conflicts: sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala --- .../spark/util/collection/OpenHashMap.scala | 17 +++++++++++++++++ .../spark/util/collection/OpenHashSet.scala | 6 ++++++ .../expressions/aggregate/Average.scala | 8 +++++++- .../catalyst/expressions/aggregate/Sum.scala | 12 ++++++++++++ .../expressions/aggregate/interfaces.scala | 8 ++++++++ .../execution/aggregate/HashAggregateExec.scala | 17 ++++++++++++----- .../compression/compressionSchemes.scala | 3 ++- .../spark/sql/execution/joins/HashJoin.scala | 4 ++-- .../sql/execution/joins/HashedRelation.scala | 9 ++++++++- .../spark/sql/execution/metric/SQLMetrics.scala | 3 ++- .../org/apache/spark/sql/hive/HiveUtils.scala | 6 +++--- 11 files changed, 79 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index 10ab0b3f89964..00cccd33daf97 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -149,6 +149,23 @@ class OpenHashMap[K : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( } } + def clear() { + // first clear the values array and value for null key + val bitSet = _keySet.getBitSet + val nullV = null.asInstanceOf[V] + val values = _values + var pos = bitSet.nextSetBit(0) + while (pos >= 0) { + values(pos) = nullV + pos = bitSet.nextSetBit(pos + 1) + } + haveNullValue = false + nullValue = nullV + _oldValues = null + // next clear the key set + _keySet.clear() + } + // The following member variables are declared as protected instead of private for the // specialization to work (specialized class extends the non-specialized one and needs access // to the "private" variables). diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index 60f6f537c1d54..835fec1320c37 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -212,6 +212,12 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( */ def nextPos(fromPos: Int): Int = _bitset.nextSetBit(fromPos) + def clear() { + _data = new Array[T](_capacity) + _bitset.clear() + _size = 0 + } + /** * Double the table's size and re-hash everything. We are not really using k, but it is declared * so Scala compiler can specialize this method (which leads to calling the specialized version diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala index 708bdbfc36058..70331bb183cd2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala @@ -53,10 +53,16 @@ case class Average(child: Expression) extends DeclarativeAggregate with Implicit } private lazy val sum = AttributeReference("sum", sumDataType)() - private lazy val count = AttributeReference("count", LongType)() + private lazy val count = AttributeReference("count", LongType, nullable = false)() override lazy val aggBufferAttributes = sum :: count :: Nil + override lazy val aggBufferAttributesForGroup: Seq[AttributeReference] = { + if (child.nullable) aggBufferAttributes + else sum.copy(nullable = false)(sum.exprId, sum.qualifier, + sum.isGenerated) :: count :: Nil + } + override lazy val initialValues = Seq( /* sum = */ Cast(Literal(0), sumDataType), /* count = */ Literal(0L) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala index 86e40a9713b36..6dacb7148072c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala @@ -53,10 +53,22 @@ case class Sum(child: Expression) extends DeclarativeAggregate with ImplicitCast override lazy val aggBufferAttributes = sum :: Nil + override lazy val aggBufferAttributesForGroup: Seq[AttributeReference] = { + if (child.nullable) aggBufferAttributes + else sum.copy(nullable = false)(sum.exprId, sum.qualifier, + sum.isGenerated) :: Nil + } + override lazy val initialValues: Seq[Expression] = Seq( /* sum = */ Literal.create(null, sumDataType) ) + override lazy val initialValuesForGroup: Seq[Expression] = Seq( + /* sum = */ + if (child.nullable) Literal.create(null, sumDataType) + else Cast(Literal(0), sumDataType) + ) + override lazy val updateExpressions: Seq[Expression] = { if (child.nullable) { Seq( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index d6d394f5b8ae9..d060fdc2b3c3c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -194,6 +194,9 @@ abstract class AggregateFunction extends Expression { /** Attributes of fields in aggBufferSchema. */ def aggBufferAttributes: Seq[AttributeReference] + /** Attributes of fields in aggBufferSchema used for group by. */ + def aggBufferAttributesForGroup: Seq[AttributeReference] = aggBufferAttributes + /** * Attributes of fields in input aggregation buffers (immutable aggregation buffers that are * merged with mutable aggregation buffers in the merge() function or merge expressions). @@ -375,6 +378,11 @@ abstract class DeclarativeAggregate */ val initialValues: Seq[Expression] + /** + * Expressions for initializing empty aggregation buffers for group by. + */ + def initialValuesForGroup: Seq[Expression] = initialValues + /** * Expressions for updating the mutable aggregation buffer based on an input row. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index cfdda6a046f80..7e09a40f01cdb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -70,6 +70,10 @@ case class HashAggregateExec( aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) } + @transient lazy private[this] val aggregateBufferAttributesForGroup = { + aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributesForGroup) + } + require(HashAggregateExec.supportsAggregate(aggregateBufferAttributes)) override lazy val allAttributes: AttributeSeq = @@ -316,7 +320,7 @@ case class HashAggregateExec( private val declFunctions = aggregateExpressions.map(_.aggregateFunction) .filter(_.isInstanceOf[DeclarativeAggregate]) .map(_.asInstanceOf[DeclarativeAggregate]) - private val bufferSchema = StructType.fromAttributes(aggregateBufferAttributes) + private val bufferSchema = StructType.fromAttributes(aggregateBufferAttributesForGroup) // The name for Fast HashMap private var fastHashMapTerm: String = _ @@ -336,7 +340,7 @@ case class HashAggregateExec( */ def createHashMap(): UnsafeFixedWidthAggregationMap = { // create initialized aggregate buffer - val initExpr = declFunctions.flatMap(f => f.initialValues) + val initExpr = declFunctions.flatMap(_.initialValuesForGroup) val initialBuffer = UnsafeProjection.create(initExpr)(EmptyRow) // create hashMap @@ -405,7 +409,7 @@ case class HashAggregateExec( val mergeExpr = declFunctions.flatMap(_.mergeExpressions) val mergeProjection = newMutableProjection( mergeExpr, - aggregateBufferAttributes ++ declFunctions.flatMap(_.inputAggBufferAttributes), + aggregateBufferAttributesForGroup ++ declFunctions.flatMap(_.inputAggBufferAttributes), subexpressionEliminationEnabled) val joinedRow = new JoinedRow() @@ -473,14 +477,14 @@ case class HashAggregateExec( } val evaluateKeyVars = evaluateVariables(keyVars) ctx.INPUT_ROW = bufferTerm - val bufferVars = aggregateBufferAttributes.zipWithIndex.map { case (e, i) => + val bufferVars = aggregateBufferAttributesForGroup.zipWithIndex.map { case (e, i) => BoundReference(i, e.dataType, e.nullable).genCode(ctx) } val evaluateBufferVars = evaluateVariables(bufferVars) // evaluate the aggregation result ctx.currentVars = bufferVars val aggResults = declFunctions.map(_.evaluateExpression).map { e => - BindReferences.bindReference(e, aggregateBufferAttributes).genCode(ctx) + BindReferences.bindReference(e, aggregateBufferAttributesForGroup).genCode(ctx) } val evaluateAggResults = evaluateVariables(aggResults) // generate the final result @@ -775,6 +779,9 @@ case class HashAggregateExec( val hashExpr = Murmur3Hash(groupingExpressions, 42) val hashEval = BindReferences.bindReference(hashExpr, child.output).genCode(ctx) + val inputAttr = aggregateBufferAttributesForGroup ++ child.output + ctx.currentVars = new Array[ExprCode](aggregateBufferAttributesForGroup.length) ++ input + val (checkFallbackForGeneratedHashMap, checkFallbackForBytesToBytesMap, resetCounter, incCounter) = if (testFallbackStartsAt.isDefined) { val countTerm = ctx.addMutableState(ctx.JAVA_INT, "fallbackCounter") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala index 00a1d54b41709..a06511c753418 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.vectorized.WritableColumnVector import org.apache.spark.sql.types._ +import org.apache.spark.util.collection.OpenHashMap private[columnar] case object PassThrough extends CompressionScheme { @@ -393,7 +394,7 @@ private[columnar] case object DictionaryEncoding extends CompressionScheme { private var values = new mutable.ArrayBuffer[T#InternalType](1024) // The dictionary that maps a value to the encoded short integer. - private val dictionary = mutable.HashMap.empty[Any, Short] + private val dictionary = new OpenHashMap[Any, Short] // Size of the serialized dictionary in bytes. Initialized to 4 since we need at least an `Int` // to store dictionary element count. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index 0396168d3f311..b73a409f5e58a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -106,9 +106,9 @@ trait HashJoin { joinRow.withLeft(srow) val matches = hashedRelation.get(joinKeys(srow)) if (matches != null) { - matches.map(joinRow.withRight(_)).filter(boundCondition) + matches.map(joinRow.withRight).filter(boundCondition) } else { - Seq.empty + Iterator.empty } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 1465346eb802d..9af1712618dca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -140,10 +140,17 @@ private[joins] class UnsafeHashedRelation( // re-used in get()/getValue() var resultRow = new UnsafeRow(numFields) + private var mapLoc = initMapLoc() + + private def initMapLoc(): BytesToBytesMap#Location = { + val map = binaryMap + new map.Location + } + override def get(key: InternalRow): Iterator[InternalRow] = { val unsafeKey = key.asInstanceOf[UnsafeRow] val map = binaryMap // avoid the compiler error - val loc = new map.Location // this could be allocated in stack + val loc = mapLoc binaryMap.safeLookup(unsafeKey.getBaseObject, unsafeKey.getBaseOffset, unsafeKey.getSizeInBytes, loc, unsafeKey.hashCode()) if (loc.isDefined) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 77b907870d678..d88763037b3bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -31,7 +31,8 @@ import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, Utils} * the executor side are automatically propagated and shown in the SQL UI through metrics. Updates * on the driver side must be explicitly posted using [[SQLMetrics.postDriverMetricUpdates()]]. */ -class SQLMetric(val metricType: String, initValue: Long = 0L) extends AccumulatorV2[Long, Long] { +final class SQLMetric(val metricType: String, initValue: Long = 0L) + extends AccumulatorV2[Long, Long] { // This is a workaround for SPARK-11013. // We may use -1 as initial value of the accumulator, if the accumulator is valid, we will // update it at the end of task and the value will be at least 0. Then we can filter out the -1 diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index c448c5a9821be..0f2b35e9afbc0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -303,9 +303,9 @@ private[spark] object HiveUtils extends Logging { if (builtinHiveVersion != hiveMetastoreVersion) { throw new IllegalArgumentException( "Builtin jars can only be used when hive execution version == hive metastore version. " + - s"Execution: $builtinHiveVersion != Metastore: $hiveMetastoreVersion. " + - s"Specify a valid path to the correct hive jars using ${HIVE_METASTORE_JARS.key} " + - s"or change ${HIVE_METASTORE_VERSION.key} to $builtinHiveVersion.") + s"Execution: $hiveExecutionVersion != Metastore: $hiveMetastoreVersion. " + + "Specify a vaild path to the correct hive jars using $HIVE_METASTORE_JARS " + + s"or change ${HIVE_METASTORE_VERSION.key} to $hiveExecutionVersion.") } // We recursively find all jars in the class loader chain, From 58af69003ef76a0a98c165542e5e9bce021bac75 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Sun, 11 Sep 2016 13:10:23 +0530 Subject: [PATCH 024/142] [SNAPPYDATA] Updated Benchmark code from Spark PR#13899 Used by the new benchmark from the PR adapted for SnappyData for its vectorized implementation. Build updated to set testOutput and other variables instead of appending to existing values (causes double append with both snappydata build adding and this adding for its tests) --- build.gradle | 12 ++++++---- .../org/apache/spark/util/Benchmark.scala | 23 +++++++++++++++---- 2 files changed, 26 insertions(+), 9 deletions(-) diff --git a/build.gradle b/build.gradle index dddce0ab57942..53d3684a80bdb 100644 --- a/build.gradle +++ b/build.gradle @@ -245,15 +245,15 @@ subprojects { def result = new StringBuilder() extensions.add(com.github.maiflai.ScalaTestAction.TESTRESULT, result) - extensions.add('testResult', { String name -> result.append(name) } ) + extensions.add('testResult', { String name -> result.setLength(0); result.append(name) } ) def output = new StringBuilder() extensions.add(com.github.maiflai.ScalaTestAction.TESTOUTPUT, output) - extensions.add('testOutput', { String name -> output.append(name) } ) + extensions.add('testOutput', { String name -> output.setLength(0); output.append(name) } ) def errorOutput = new StringBuilder() extensions.add(com.github.maiflai.ScalaTestAction.TESTERROR, errorOutput) - extensions.add('testError', { String name -> errorOutput.append(name) } ) + extensions.add('testError', { String name -> errorOutput.setLength(0); errorOutput.append(name) } ) // running a single scala suite if (rootProject.hasProperty('singleSuite')) { @@ -295,8 +295,10 @@ subprojects { test.configure { onlyIf { ! Boolean.getBoolean('skip.tests') } - jvmArgs '-XX:+HeapDumpOnOutOfMemoryError', '-XX:MaxPermSize=512m', '-XX:ReservedCodeCacheSize=512m' - maxHeapSize '3g' + jvmArgs '-ea', '-XX:+HeapDumpOnOutOfMemoryError','-XX:+UseConcMarkSweepGC', + '-XX:+UseParNewGC', '-XX:+CMSClassUnloadingEnabled', '-XX:MaxPermSize=512m' + minHeapSize '4g' + maxHeapSize '4g' // disable assertions for hive tests as in Spark's pom.xml because HiveCompatibilitySuite currently fails (SPARK-4814) if (test.project.name.contains('snappy-spark-hive_')) { jvmArgs '-da' diff --git a/core/src/main/scala/org/apache/spark/util/Benchmark.scala b/core/src/main/scala/org/apache/spark/util/Benchmark.scala index 7def44bd2a2b1..7576faa99c96d 100644 --- a/core/src/main/scala/org/apache/spark/util/Benchmark.scala +++ b/core/src/main/scala/org/apache/spark/util/Benchmark.scala @@ -69,12 +69,17 @@ private[spark] class Benchmark( * @param name of the benchmark case * @param numIters if non-zero, forces exactly this many iterations to be run */ - def addCase(name: String, numIters: Int = 0)(f: Int => Unit): Unit = { - addTimerCase(name, numIters) { timer => + def addCase( + name: String, + numIters: Int = 0, + prepare: () => Unit = () => { }, + cleanup: () => Unit = () => { })(f: Int => Unit): Unit = { + val timedF = (timer: Benchmark.Timer) => { timer.startTiming() f(timer.iteration) timer.stopTiming() } + benchmarks += Benchmark.Case(name, timedF, numIters, prepare, cleanup) } /** @@ -101,7 +106,12 @@ private[spark] class Benchmark( val results = benchmarks.map { c => println(" Running case: " + c.name) - measure(valuesPerIteration, c.numIters)(c.fn) + try { + c.prepare() + measure(valuesPerIteration, c.numIters)(c.fn) + } finally { + c.cleanup() + } } println @@ -188,7 +198,12 @@ private[spark] object Benchmark { } } - case class Case(name: String, fn: Timer => Unit, numIters: Int) + case class Case( + name: String, + fn: Timer => Unit, + numIters: Int, + prepare: () => Unit = () => { }, + cleanup: () => Unit = () => { }) case class Result(avgMs: Double, bestRate: Double, bestMs: Double) /** From 1c4ff5a42441fb107a101d47f15cf03cdc76d5ba Mon Sep 17 00:00:00 2001 From: Amogh Shetkar Date: Tue, 20 Sep 2016 16:40:35 +0530 Subject: [PATCH 025/142] [SNAPPYDATA] Spark version 2.0.1-2 --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 53d3684a80bdb..fed74f727af00 100644 --- a/build.gradle +++ b/build.gradle @@ -45,7 +45,7 @@ allprojects { apply plugin: 'idea' group = 'io.snappydata' - version = '2.0.1-1' + version = '2.0.1-2' ext { scalaBinaryVersion = '2.11' From 5acb3599bbce373880ff495e159be6641c204417 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Thu, 22 Sep 2016 18:38:06 +0530 Subject: [PATCH 026/142] [SNAPPYDATA] fixing antlr generated code for IDEA --- sql/catalyst/.gitignore | 1 + sql/catalyst/build.gradle | 5 ++++- 2 files changed, 5 insertions(+), 1 deletion(-) create mode 100644 sql/catalyst/.gitignore diff --git a/sql/catalyst/.gitignore b/sql/catalyst/.gitignore new file mode 100644 index 0000000000000..42b6ce41f8a60 --- /dev/null +++ b/sql/catalyst/.gitignore @@ -0,0 +1 @@ +src/generated/antlr4 diff --git a/sql/catalyst/build.gradle b/sql/catalyst/build.gradle index d2deeb94b9b93..cc0e9bbf2822e 100644 --- a/sql/catalyst/build.gradle +++ b/sql/catalyst/build.gradle @@ -38,10 +38,13 @@ compileScala.dependsOn generateGrammarSource sourceSets.main.antlr.srcDirs = [ 'src/main/antlr4' ] +// use an output directory that IDEA can easily find +String antlrOut = 'src/generated/antlr4' // add generated sources to scala compiler path (plugin adds it to java path) -sourceSets.main.scala.srcDir generateGrammarSource.outputDirectory +sourceSets.main.scala.srcDir antlrOut sourceSets.main.java.srcDirs = [] generateGrammarSource { arguments += [ '-package', 'org.apache.spark.sql.catalyst.parser', '-visitor' ] + outputDirectory = file(antlrOut) } From 26adf26d012d62dcc60574e641c4ebf03d09fe57 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Tue, 18 Oct 2016 04:35:21 +0530 Subject: [PATCH 027/142] [SNAP-1083] fix numBuckets handling (#15) - don't apply numBuckets in Shuffle partitioning since Shuffle cannot create a compatible partitioning with matching numBuckets (only numPartitions) - check numBuckets too in HashPartitioning compatibility Conflicts: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala --- .../plans/physical/partitioning.scala | 11 ++--- .../exchange/EnsureRequirements.scala | 45 +++++++++++++------ 2 files changed, 38 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index e488281de0dc1..dd12426d17bf4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -265,9 +265,8 @@ case class OrderlessHashPartitioning(expressions: Seq[Expression], numPartitions * in the same partition. Moreover while evaluating expressions if they are given in different order * than this partitioning then also it is considered equal. */ -case class OrderlessHashPartitioning(expressions: Seq[Expression], numPartitions: Int) - extends Expression with Partitioning with Unevaluable { - +case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int, + numBuckets: Int = 0) extends Expression with Partitioning with Unevaluable { override def children: Seq[Expression] = expressions override def nullable: Boolean = false @@ -291,12 +290,14 @@ case class OrderlessHashPartitioning(expressions: Seq[Expression], numPartitions } override def compatibleWith(other: Partitioning): Boolean = other match { - case p: HashPartitioning => anyOrderEquals(p) + case o: HashPartitioning => + this.numBuckets == o.numBuckets && anyOrderEquals(o) case _ => false } override def guarantees(other: Partitioning): Boolean = other match { - case o: HashPartitioning => anyOrderEquals(o) + case o: HashPartitioning => + this.numBuckets == o.numBuckets && this.anyOrderEquals(o) case _ => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index e3d28388c5470..6b398640e1e56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -179,19 +179,38 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { numPartitionsSet.headOption } - val targetNumPartitions = requiredNumPartitions.getOrElse(childrenNumPartitions.max) - - children = children.zip(requiredChildDistributions).zipWithIndex.map { - case ((child, distribution), index) if childrenIndexes.contains(index) => - if (child.outputPartitioning.numPartitions == targetNumPartitions) { - child - } else { - val defaultPartitioning = distribution.createPartitioning(targetNumPartitions) - child match { - // If child is an exchange, we replace it with a new one having defaultPartitioning. - case ShuffleExchangeExec(_, c, _) => ShuffleExchangeExec(defaultPartitioning, c) - case _ => ShuffleExchangeExec(defaultPartitioning, child) - } + children = if (useExistingPartitioning) { + // We do not need to shuffle any child's output. + children + } else { + // We need to shuffle at least one child's output. + // Now, we will determine the number of partitions that will be used by created + // partitioning schemes. + val numPartitions = { + // Let's see if we need to shuffle all child's outputs when we use + // maxChildrenNumPartitions. + val shufflesAllChildren = children.zip(requiredChildDistributions).forall { + case (child, distribution) => + !child.outputPartitioning.guarantees( + createPartitioning(distribution, maxChildrenNumPartitions)) + } + // If we need to shuffle all children, we use defaultNumPreShufflePartitions as the + // number of partitions. Otherwise, we use maxChildrenNumPartitions. + if (shufflesAllChildren) defaultNumPreShufflePartitions else maxChildrenNumPartitions + } + children.zip(requiredChildDistributions).map { + case (child, distribution) => + val targetPartitioning = createPartitioning(distribution, + numPartitions) + if (child.outputPartitioning.guarantees(targetPartitioning)) { + child + } else { + child match { + // If child is an exchange, we replace it with + // a new one having targetPartitioning. + case ShuffleExchange(_, c, _) => ShuffleExchange(targetPartitioning, c) + case _ => ShuffleExchange(targetPartitioning, child) + } } case ((child, _), _) => child From 5cacaa1fe787ab0e561d434a24b9bad7ff718de2 Mon Sep 17 00:00:00 2001 From: Amogh Shetkar Date: Thu, 20 Oct 2016 21:39:25 +0530 Subject: [PATCH 028/142] [SNAPPYDATA] Spark version 2.0.1-3 --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index fed74f727af00..5f3c2fa470c40 100644 --- a/build.gradle +++ b/build.gradle @@ -45,7 +45,7 @@ allprojects { apply plugin: 'idea' group = 'io.snappydata' - version = '2.0.1-2' + version = '2.0.1-3' ext { scalaBinaryVersion = '2.11' From 2142c81fd98df16ab6bc2b785d80daa189c3a5c8 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Tue, 25 Oct 2016 00:44:58 +0530 Subject: [PATCH 029/142] [SNAPPYDATA] updating snappy-spark version after the merge --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 5f3c2fa470c40..e7dd3cda9e7e9 100644 --- a/build.gradle +++ b/build.gradle @@ -45,7 +45,7 @@ allprojects { apply plugin: 'idea' group = 'io.snappydata' - version = '2.0.1-3' + version = '2.0.2-1' ext { scalaBinaryVersion = '2.11' From ce30bd98ec0e63f0e226e10016b67b86f3b6550d Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Thu, 24 Nov 2016 19:45:40 +0530 Subject: [PATCH 030/142] [SNAPPYDATA] Bumping version to 2.0.3-1 --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index e7dd3cda9e7e9..402593595ce2a 100644 --- a/build.gradle +++ b/build.gradle @@ -45,7 +45,7 @@ allprojects { apply plugin: 'idea' group = 'io.snappydata' - version = '2.0.2-1' + version = '2.0.3-1' ext { scalaBinaryVersion = '2.11' From 46c3807cea572540ca41262e6a7735b1be4133a5 Mon Sep 17 00:00:00 2001 From: Rishitesh Mishra Date: Sun, 27 Nov 2016 19:54:49 +0530 Subject: [PATCH 031/142] [SNAPPYDATA] Made two methods in Executor as protected to make them customizable for SnappyExecutors. (#26) Conflicts: core/src/main/scala/org/apache/spark/executor/Executor.scala --- core/src/main/scala/org/apache/spark/executor/Executor.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index b3b107e5a49fd..4f1b35415e5af 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -126,7 +126,7 @@ private[spark] class Executor( // Create our ClassLoader // do this after SparkEnv creation so can access the SecurityManager - private val urlClassLoader = createClassLoader() + protected val urlClassLoader = createClassLoader() private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader) // Set the classloader for serializer @@ -748,7 +748,7 @@ private[spark] class Executor( * Download any missing dependencies if we receive a new set of files and JARs from the * SparkContext. Also adds any new JARs we fetched to the class loader. */ - private def updateDependencies(newFiles: Map[String, Long], newJars: Map[String, Long]) { + protected def updateDependencies(newFiles: HashMap[String, Long], newJars: HashMap[String, Long]) { lazy val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) synchronized { // Fetch missing dependencies From 1c255df34050d7a33e6ede5dc7784ff2c665acbc Mon Sep 17 00:00:00 2001 From: Soubhik Chakraborty Date: Mon, 28 Nov 2016 15:53:28 +0530 Subject: [PATCH 032/142] [SNAPPYDATA]: Honoring JAVA_HOME variable while compiling java files instead of using system javac. This eliminates problem when system jdk is set differently from JAVA_HOME --- common/sketch/build.gradle | 2 +- common/unsafe/build.gradle | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/common/sketch/build.gradle b/common/sketch/build.gradle index a5e5efff08b5a..28936ece117e9 100644 --- a/common/sketch/build.gradle +++ b/common/sketch/build.gradle @@ -24,5 +24,5 @@ dependencies { tasks.withType(JavaCompile) { options.compilerArgs << '-XDignore.symbol.file' options.fork = true - options.forkOptions.executable = 'javac' + options.forkOptions.executable = "${System.properties['java.home']}/../bin/javac" } diff --git a/common/unsafe/build.gradle b/common/unsafe/build.gradle index 69d29942f5f1c..ee2347c9eb872 100644 --- a/common/unsafe/build.gradle +++ b/common/unsafe/build.gradle @@ -34,5 +34,5 @@ sourceSets.main.java.srcDirs = [ 'src/main/java' ] tasks.withType(JavaCompile) { options.compilerArgs << '-XDignore.symbol.file' options.fork = true - options.forkOptions.executable = 'javac' + options.forkOptions.executable = "${System.properties['java.home']}/../bin/javac" } From 455f3281edcf4001d85d69de1c67895445bf1b9e Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Thu, 1 Dec 2016 21:10:12 +0530 Subject: [PATCH 033/142] [SNAP-1198] Use ConcurrentHashMap instead of queue for ContextCleaner.referenceBuffer (#32) Use a map instead of queue for ContextCleaner.referenceBuffer. Profiling shows lot of time being spent removing from queue where a hash map will do (referenceQueue is already present for poll). Conflicts: core/src/main/scala/org/apache/spark/ContextCleaner.scala --- core/src/main/scala/org/apache/spark/ContextCleaner.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 4d884dec07916..04781a2945786 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -64,7 +64,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { * have not been handled by the reference queue. */ private val referenceBuffer = - Collections.newSetFromMap[CleanupTaskWeakReference](new ConcurrentHashMap) + new ConcurrentHashMap[CleanupTaskWeakReference, java.lang.Boolean]() private val referenceQueue = new ReferenceQueue[AnyRef] @@ -171,7 +171,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { /** Register an object for cleanup. */ private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask): Unit = { - referenceBuffer.add(new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue)) + referenceBuffer.put(new CleanupTaskWeakReference(task, objectForCleanup, + referenceQueue), java.lang.Boolean.TRUE) } /** Keep cleaning RDD, shuffle, and broadcast state. */ From 17b6b3b34ee55a19963c5b27642e0494f4cd4628 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Sat, 3 Dec 2016 17:35:11 +0530 Subject: [PATCH 034/142] [SNAP-1194] explicit addLong/longValue methods in SQLMetrics (#33) This avoids runtime erasure for add/value methods that will result in unnecessary boxing/unboxing overheads. - Adding spark-kafka-sql project - Update version of deps as per upstream. - corrected kafka-clients reference Conflicts: sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala --- assembly/build.gradle | 1 + core/build.gradle | 2 +- external/kafka-0-10-sql/build.gradle | 33 +++++++++++++++++++ external/kafka-0-10/build.gradle | 3 +- settings.gradle | 2 ++ .../execution/basicPhysicalOperators.scala | 6 ++-- .../sql/execution/metric/SQLMetrics.scala | 5 +++ 7 files changed, 47 insertions(+), 5 deletions(-) create mode 100644 external/kafka-0-10-sql/build.gradle diff --git a/assembly/build.gradle b/assembly/build.gradle index 63db32e3e41f6..0d81f789eb05b 100644 --- a/assembly/build.gradle +++ b/assembly/build.gradle @@ -27,6 +27,7 @@ dependencies { compile project(subprojectBase + 'snappy-spark-streaming_' + scalaBinaryVersion) compile project(subprojectBase + 'snappy-spark-streaming-kafka-0.8_' + scalaBinaryVersion) compile project(subprojectBase + 'snappy-spark-streaming-kafka-0.10_' + scalaBinaryVersion) + compile project(subprojectBase + 'snappy-spark-sql-kafka-0.10_' + scalaBinaryVersion) compile project(subprojectBase + 'snappy-spark-yarn_' + scalaBinaryVersion) compile project(subprojectBase + 'snappy-spark-mllib_' + scalaBinaryVersion) compile project(subprojectBase + 'snappy-spark-graphx_' + scalaBinaryVersion) diff --git a/core/build.gradle b/core/build.gradle index 9395a129dac33..1caee72201e40 100644 --- a/core/build.gradle +++ b/core/build.gradle @@ -124,7 +124,7 @@ dependencies { } compile group: 'org.apache.ivy', name: 'ivy', version: '2.4.0' compile group: 'oro', name: 'oro', version: '2.0.8' - compile(group: 'net.razorvine', name: 'pyrolite', version: '4.9') { + compile(group: 'net.razorvine', name: 'pyrolite', version: '4.13') { exclude(group: 'net.razorvine', module: 'serpent') } compile group: 'net.sf.py4j', name: 'py4j', version: '0.10.1' diff --git a/external/kafka-0-10-sql/build.gradle b/external/kafka-0-10-sql/build.gradle new file mode 100644 index 0000000000000..45108783e735d --- /dev/null +++ b/external/kafka-0-10-sql/build.gradle @@ -0,0 +1,33 @@ +/* + * Copyright (c) 2016 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ + +description = 'Kafka 0.10 Source for Structured Streaming' + +dependencies { + compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) + provided project(subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion) + provided project(subprojectBase + 'snappy-spark-catalyst_' + scalaBinaryVersion) + provided project(subprojectBase + 'snappy-spark-sql_' + scalaBinaryVersion) + + compile group: 'org.apache.kafka', name: 'kafka-clients', version: '0.10.0.1' + + testCompile project(path: subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion, configuration: 'testOutput') + testCompile project(path: subprojectBase + 'snappy-spark-catalyst_' + scalaBinaryVersion, configuration: 'testOutput') + testCompile project(path: subprojectBase + 'snappy-spark-sql_' + scalaBinaryVersion, configuration: 'testOutput') + testCompile group: 'org.apache.kafka', name: 'kafka_' + scalaBinaryVersion, version: '0.10.0.1' + testCompile group: 'net.sf.jopt-simple', name: 'jopt-simple', version: '3.2' +} diff --git a/external/kafka-0-10/build.gradle b/external/kafka-0-10/build.gradle index d5dc6611a97f0..9b6db7851f1d4 100644 --- a/external/kafka-0-10/build.gradle +++ b/external/kafka-0-10/build.gradle @@ -19,7 +19,8 @@ description = 'Spark Integration for Kafka 0.10' dependencies { compile project(subprojectBase + 'snappy-spark-tags_' + scalaBinaryVersion) - compile project(subprojectBase + 'snappy-spark-streaming_' + scalaBinaryVersion) + provided project(subprojectBase + 'snappy-spark-core_' + scalaBinaryVersion) + provided project(subprojectBase + 'snappy-spark-streaming_' + scalaBinaryVersion) compile(group: 'org.apache.kafka', name: 'kafka_' + scalaBinaryVersion, version: '0.10.0.1') { exclude(group: 'com.sun.jmx', module: 'jmxri') diff --git a/settings.gradle b/settings.gradle index ca33d18d94bf5..7150a225b3de3 100644 --- a/settings.gradle +++ b/settings.gradle @@ -40,6 +40,7 @@ include ':snappy-spark-streaming-flume_' + scalaBinaryVersion include ':snappy-spark-streaming-flume-sink_' + scalaBinaryVersion include ':snappy-spark-streaming-kafka-0.8_' + scalaBinaryVersion include ':snappy-spark-streaming-kafka-0.10_' + scalaBinaryVersion +include ':snappy-spark-sql-kafka-0.10_' + scalaBinaryVersion include ':snappy-spark-examples_' + scalaBinaryVersion include ':snappy-spark-repl_' + scalaBinaryVersion include ':snappy-spark-launcher_' + scalaBinaryVersion @@ -67,6 +68,7 @@ project(':snappy-spark-streaming-flume_' + scalaBinaryVersion).projectDir = "$ro project(':snappy-spark-streaming-flume-sink_' + scalaBinaryVersion).projectDir = "$rootDir/external/flume-sink" as File project(':snappy-spark-streaming-kafka-0.8_' + scalaBinaryVersion).projectDir = "$rootDir/external/kafka-0-8" as File project(':snappy-spark-streaming-kafka-0.10_' + scalaBinaryVersion).projectDir = "$rootDir/external/kafka-0-10" as File +project(':snappy-spark-sql-kafka-0.10_' + scalaBinaryVersion).projectDir = "$rootDir/external/kafka-0-10-sql" as File project(':snappy-spark-examples_' + scalaBinaryVersion).projectDir = "$rootDir/examples" as File project(':snappy-spark-repl_' + scalaBinaryVersion).projectDir = "$rootDir/repl" as File project(':snappy-spark-launcher_' + scalaBinaryVersion).projectDir = "$rootDir/launcher" as File diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index a15a8d11aa2a0..ece8bcd9681d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -202,7 +202,7 @@ case class FilterExec(condition: Expression, child: SparkPlan) |do { | $generated | $nullChecks - | $numOutput.add(1); + | $numOutput.addLong(1); | ${consume(ctx, resultVars)} |} while(false); """.stripMargin @@ -308,7 +308,7 @@ case class SampleExec( s""" | int $samplingCount = $sampler.sample(); | while ($samplingCount-- > 0) { - | $numOutput.add(1); + | $numOutput.addLong(1); | ${consume(ctx, input)} | } """.stripMargin.trim @@ -322,7 +322,7 @@ case class SampleExec( s""" | if ($sampler.sample() != 0) { - | $numOutput.add(1); + | $numOutput.addLong(1); | ${consume(ctx, input)} | } """.stripMargin.trim diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index d88763037b3bf..67bb66da5f38b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -64,6 +64,11 @@ final class SQLMetric(val metricType: String, initValue: Long = 0L) def set(v: Long): Unit = _value = v + // avoid the runtime generic Object conversion of add(), value() + final def addLong(v: Long): Unit = _value += v + + final def longValue: Long = _value + def +=(v: Long): Unit = _value += v override def value: Long = _value From 98b2f85eb5523cde9236ce13bffd50e5d0d71b3b Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Thu, 24 Nov 2016 14:44:51 +0530 Subject: [PATCH 035/142] [SNAPPYDATA] More optimizations to UTF8String - allow direct UTF8String objects in RDD data conversions to DataFrame; new UTF8String.cloneIfRequired to clone only if required used by above - allow for some precision change in QueryTest result comparison --- .../apache/spark/unsafe/types/UTF8String.java | 30 +++++++++++++++---- .../sql/catalyst/JavaTypeInference.scala | 4 +++ .../spark/sql/catalyst/ScalaReflection.scala | 10 +++++++ .../org/apache/spark/sql/QueryTest.scala | 1 + 4 files changed, 40 insertions(+), 5 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index b0d0c44823e68..712129391cd80 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -1183,7 +1183,27 @@ public String toString() { @Override public UTF8String clone() { - return fromBytes(getBytes()); + UTF8String newString = fromBytes(getBytes()); + if (isAscii) { + newString.isAscii = true; + } + return newString; + } + + public UTF8String cloneIfRequired() { + if (offset == BYTE_ARRAY_OFFSET && + ((byte[])base).length == numBytes) { + return this; + } else { + final int numBytes = this.numBytes; + final byte[] bytes = new byte[numBytes]; + copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, numBytes); + UTF8String newString = fromAddress(bytes, BYTE_ARRAY_OFFSET, numBytes); + if (isAscii) { + newString.isAscii = true; + } + return newString; + } } public UTF8String copy() { @@ -1194,6 +1214,10 @@ public UTF8String copy() { @Override public int compareTo(@Nonnull final UTF8String other) { + return compare(other); + } + + public int compare(final UTF8String other) { int len = Math.min(numBytes, other.numBytes); int wordMax = (len / 8) * 8; long roffset = other.offset; @@ -1219,10 +1243,6 @@ public int compareTo(@Nonnull final UTF8String other) { return numBytes - other.numBytes; } - public int compare(final UTF8String other) { - return compareTo(other); - } - @Override public boolean equals(final Object other) { if (other instanceof UTF8String) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index 3ecc137c8cd7f..a43381c05ad7e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -240,6 +240,8 @@ object JavaTypeInference { case c if c == classOf[java.lang.String] => Invoke(getPath, "toString", ObjectType(classOf[String])) + case c if c == classOf[UTF8String] => + Invoke(getPath, "toString", ObjectType(classOf[String])) case c if c == classOf[java.math.BigDecimal] => Invoke(getPath, "toJavaBigDecimal", ObjectType(classOf[java.math.BigDecimal])) @@ -421,6 +423,8 @@ object JavaTypeInference { Invoke(inputObject, "floatValue", FloatType) case c if c == classOf[java.lang.Double] => Invoke(inputObject, "doubleValue", DoubleType) + case c if c == classOf[UTF8String] => + Invoke(inputObject, "cloneIfRequired", StringType) case _ if typeToken.isArray => toCatalystArray(inputObject, typeToken.getComponentType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 9a4bf0075a178..cd34a18754377 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -263,6 +263,9 @@ object ScalaReflection extends ScalaReflection { case t if t <:< localTypeOf[java.lang.String] => Invoke(getPath, "toString", ObjectType(classOf[String]), returnNullable = false) + case t if t <:< localTypeOf[UTF8String] => + Invoke(getPath, "cloneIfRequired", ObjectType(classOf[UTF8String])) + case t if t <:< localTypeOf[java.math.BigDecimal] => Invoke(getPath, "toJavaBigDecimal", ObjectType(classOf[java.math.BigDecimal]), returnNullable = false) @@ -534,6 +537,12 @@ object ScalaReflection extends ScalaReflection { inputObject :: Nil, returnNullable = false) + case t if t <:< localTypeOf[UTF8String] => + Invoke( + inputObject, + "cloneIfRequired", + StringType) + case t if t <:< localTypeOf[java.sql.Timestamp] => StaticInvoke( DateTimeUtils.getClass, @@ -745,6 +754,7 @@ object ScalaReflection extends ScalaReflection { val Schema(dataType, nullable) = schemaFor(elementType) Schema(ArrayType(dataType, containsNull = nullable), nullable = true) case t if t <:< localTypeOf[String] => Schema(StringType, nullable = true) + case t if t <:< localTypeOf[UTF8String] => Schema(StringType, nullable = true) case t if t <:< localTypeOf[java.sql.Timestamp] => Schema(TimestampType, nullable = true) case t if t <:< localTypeOf[java.sql.Date] => Schema(DateType, nullable = true) case t if t <:< localTypeOf[BigDecimal] => Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 9fb8be423614b..73949c682780b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -290,6 +290,7 @@ object QueryTest { Row.fromSeq(row.toSeq.map { case null => null case d: java.math.BigDecimal => BigDecimal(d) + case d: Double => math.floor(d * 1000.0 + 0.5) // round to three digits // Convert array to Seq for easy equality check. case b: Array[_] => b.toSeq case r: Row => prepareRow(r) From 9a566cc796dc91f21c7a0957db0de62e590a50c2 Mon Sep 17 00:00:00 2001 From: Hemant Bhanawat Date: Fri, 9 Dec 2016 15:41:03 +0530 Subject: [PATCH 036/142] [SNAPPYDATA] Adding fixed stats to common filter expressions Missing filter statistics in filter's logical plan is causing incorrect plan selection at times. Also, join statistics always return sizeInBytes as the product of its child sizeInBytes which result in a big number. For join, product makes sense only when it is a cartesian product join. Hence, fixed the spark code to check for the join type. If the join is a equi-join, we now sum the sizeInBytes of the child instead of doing a product. For missing filter statistics, adding a heuristics based sizeInBytes calculation mentioned below. If the filtering condition is: - equal to: sizeInBytes is 5% of the child sizeInBytes - greater than less than: sizeInBytes is 50% of the child sizeInBytes - isNull: sizeInBytes is 50% of the child sizeInBytes - starts with: sizeInBytes is 10% of the child sizeInBytes Conflicts: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala --- .../plans/logical/basicLogicalOperators.scala | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index a4fca790dd086..253a5d5e81223 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} @@ -130,6 +131,19 @@ case class Filter(condition: Expression, child: LogicalPlan) override def maxRows: Option[Long] = child.maxRows + override lazy val statistics: Statistics = { + // Expected filtering by expressions + val expectedFilterDivisor = condition match { + case EqualTo(_, _) => 20 + case StartsWith(_, _) => 10 + case LessThan(_, _) | LessThanOrEqual(_, _) | + GreaterThan(_, _) | GreaterThanOrEqual(_, _) => 2 + case IsNull(_) => 2 + case _ => 1 + } + child.statistics.copy(sizeInBytes = child.statistics.sizeInBytes / expectedFilterDivisor) + } + override protected def validConstraints: Set[Expression] = { val predicates = splitConjunctivePredicates(condition) .filterNot(SubqueryExpression.hasCorrelatedSubquery) @@ -341,6 +355,18 @@ case class Join( case UsingJoin(_, _) => false case _ => resolvedExceptNatural } + + override lazy val statistics: Statistics = joinType match { + case LeftAnti | LeftSemi => + // LeftSemi and LeftAnti won't ever be bigger than left + left.statistics.copy() + case _ if ExtractEquiJoinKeys.unapply(this).isDefined => + Statistics(sizeInBytes = children.map(_.statistics.sizeInBytes).sum) + case _ => + // make sure we don't propagate isBroadcastable in other joins, because + // they could explode the size. + super.statistics.copy(isBroadcastable = false) + } } /** From f27ef92a5b24ebf939200a217c6669e322a575fb Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Sat, 10 Dec 2016 00:55:19 +0530 Subject: [PATCH 037/142] [SNAPPYDATA] adding kryo serialization missing in LongHashedRelation Conflicts: sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala --- .../spark/sql/execution/joins/HashedRelation.scala | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 9af1712618dca..678262ada8bee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -789,7 +789,8 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap private[joins] class LongHashedRelation( private var nFields: Int, - private var map: LongToUnsafeRowMap) extends HashedRelation with Externalizable { + private var map: LongToUnsafeRowMap) extends HashedRelation + with Externalizable with KryoSerializable { private var resultRow: UnsafeRow = new UnsafeRow(nFields) @@ -831,6 +832,11 @@ private[joins] class LongHashedRelation( out.writeObject(map) } + override def write(kryo: Kryo, output: Output): Unit = { + output.writeInt(nFields) + kryo.writeClassAndObject(output, map) + } + override def readExternal(in: ObjectInput): Unit = { nFields = in.readInt() resultRow = new UnsafeRow(nFields) @@ -838,6 +844,12 @@ private[joins] class LongHashedRelation( } override def getAverageProbesPerLookup: Double = map.getAverageProbesPerLookup + + override def read(kryo: Kryo, input: Input): Unit = { + nFields = input.readInt() + resultRow = new UnsafeRow(nFields) + map = kryo.readClassAndObject(input).asInstanceOf[LongToUnsafeRowMap] + } } /** From 6951cda7449b9f89f0032ebe9f990131078d5466 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Sat, 10 Dec 2016 21:58:00 +0530 Subject: [PATCH 038/142] [SNAPPYDATA] Correcting HashPartitioning interface to match apache spark Addition of numBuckets as default parameter made HashPartitioning incompatible with upstream apache spark. Now adding it separately so restore compatibility. Conflicts: core/src/main/scala/org/apache/spark/Partitioner.scala sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala --- .../scala/org/apache/spark/Partitioner.scala | 6 ++---- .../catalyst/plans/physical/partitioning.scala | 16 ++++++++++++++-- .../execution/exchange/ShuffleExchangeExec.scala | 1 + 3 files changed, 17 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index bdfecd08f27c8..7d0e7591a1954 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -106,14 +106,12 @@ object Partitioner { * so attempting to partition an RDD[Array[_]] or RDD[(Array[_], _)] using a HashPartitioner will * produce an unexpected or incorrect result. */ -class HashPartitioner(partitions: Int) extends Partitioner { +class HashPartitioner(partitions: Int, buckets: Int) extends Partitioner { require(partitions >= 0, s"Number of partitions ($partitions) cannot be negative.") require(buckets >= 0, s"Number of buckets ($buckets) cannot be negative.") - def this(partitions: Int) { - this(partitions, 0) - } + def this(partitions: Int) = this(partitions, 0) def numPartitions: Int = partitions diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index dd12426d17bf4..08841c2419c35 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -265,8 +265,10 @@ case class OrderlessHashPartitioning(expressions: Seq[Expression], numPartitions * in the same partition. Moreover while evaluating expressions if they are given in different order * than this partitioning then also it is considered equal. */ -case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int, - numBuckets: Int = 0) extends Expression with Partitioning with Unevaluable { +case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) + extends Expression with Partitioning with Unevaluable { + + private[sql] var numBuckets: Int = 0 override def children: Seq[Expression] = expressions override def nullable: Boolean = false @@ -337,6 +339,16 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) def partitionIdExpression: Expression = Pmod(new Murmur3Hash(expressions), Literal(numPartitions)) } +object HashPartitioning { + + def apply(expressions: Seq[Expression], numPartitions: Int, + numBuckets: Int): HashPartitioning = { + val partitioning = HashPartitioning(expressions, numPartitions) + partitioning.numBuckets = numBuckets + partitioning + } +} + /** * Represents a partitioning where rows are split across partitions based on some total ordering of * the expressions specified in `ordering`. When data is partitioned in this manner the following diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index 4d95ee34f30de..297f46142f0cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -214,6 +214,7 @@ object ShuffleExchangeExec { // `HashPartitioning.partitionIdExpression` to produce partitioning key. override def getPartition(key: Any): Int = key.asInstanceOf[Int] } + // case p@HashPartitioning(_, n) => new HashPartitioner(n, p.numBuckets) case RangePartitioning(sortingExpressions, numPartitions) => // Internally, RangePartitioner runs a job on the RDD that samples keys to compute // partition bounds. To get accurate samples, we need to copy the mutable keys. From b0400cd545c014f0e6b250a2a9be44b08c6c4abe Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Sun, 11 Dec 2016 16:32:33 +0530 Subject: [PATCH 039/142] [SNAP-1233] clear InMemorySorter before calling its reset (#35) This is done so that any spill call (due to no EVICTION_DOWN) from within the spill call will return without doing anything, else it results in NPE trying to read page tables which have already been cleared. --- .../collection/unsafe/sort/UnsafeExternalSorter.java | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 66118f454159b..4d682dccff9c6 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -226,7 +226,17 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { // Note that this is more-or-less going to be a multiple of the page size, so wasted space in // pages will currently be counted as memory spilled even though that space isn't actually // written to disk. This also counts the space needed to store the sorter's pointer array. - inMemSorter.reset(); + + // temporarily clear inMemorySorter so that a recursive spill call will return + final UnsafeInMemorySorter memSorter = inMemSorter; + if (memSorter != null) { + inMemSorter = null; + try { + memSorter.reset(); + } finally { + inMemSorter = memSorter; + } + } // Reset the in-memory sorter's pointer array only after freeing up the memory pages holding the // records. Otherwise, if the task is over allocated memory, then without freeing the memory // pages, we might not be able to get memory for the pointer array. From 1f46a571fbee08493a04bd70c900831bc404ff53 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Mon, 12 Dec 2016 05:37:07 +0530 Subject: [PATCH 040/142] [SNAPPYDATA] Adding more filter conditions for plan sizing as followup - IN is 50% of original - StartsWith, EndsWith 10% - Contains and LIKE at 20% - AND is multiplication of sizing of left and right (with max filtering of 5%) - OR is 1/x+1/y sizing of the left and right (with min filtering of 50%) - NOT three times of that without NOT --- .../plans/logical/basicLogicalOperators.scala | 21 ++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 253a5d5e81223..652405c955b3f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -132,16 +132,27 @@ case class Filter(condition: Expression, child: LogicalPlan) override def maxRows: Option[Long] = child.maxRows override lazy val statistics: Statistics = { - // Expected filtering by expressions - val expectedFilterDivisor = condition match { + // Expected filtering by expressions based on some constants for now. + def expectedFilterDivisor(cond: Expression): Int = cond match { case EqualTo(_, _) => 20 - case StartsWith(_, _) => 10 case LessThan(_, _) | LessThanOrEqual(_, _) | GreaterThan(_, _) | GreaterThanOrEqual(_, _) => 2 - case IsNull(_) => 2 + case In(_, _) => 2 + case StartsWith(_, _) | EndsWith(_, _) => 10 + case Contains(_, _) | Like(_, _) => 5 + case And(left, right) => + math.min(20, expectedFilterDivisor(left) * expectedFilterDivisor(right)) + case Or(left, right) => + val leftDivisor = expectedFilterDivisor(left) + val rightDivisor = expectedFilterDivisor(right) + math.max(2, (leftDivisor * rightDivisor) / (leftDivisor + rightDivisor)) + case Not(e) => math.max(2, expectedFilterDivisor(e) / 3) + case IsNull(_) => 3 case _ => 1 } - child.statistics.copy(sizeInBytes = child.statistics.sizeInBytes / expectedFilterDivisor) + + child.statistics.copy(sizeInBytes = child.statistics.sizeInBytes / + expectedFilterDivisor(condition)) } override protected def validConstraints: Set[Expression] = { From 66d9d42b36fed73f2ee9085c466ab0dc91162275 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Tue, 13 Dec 2016 23:02:01 +0530 Subject: [PATCH 041/142] [SNAPPYDATA] reduced factors in filters a bit to be more conservative --- .../catalyst/plans/logical/basicLogicalOperators.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 652405c955b3f..5e41a0a6edc5f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -134,19 +134,19 @@ case class Filter(condition: Expression, child: LogicalPlan) override lazy val statistics: Statistics = { // Expected filtering by expressions based on some constants for now. def expectedFilterDivisor(cond: Expression): Int = cond match { - case EqualTo(_, _) => 20 + case EqualTo(_, _) => 10 case LessThan(_, _) | LessThanOrEqual(_, _) | GreaterThan(_, _) | GreaterThanOrEqual(_, _) => 2 case In(_, _) => 2 - case StartsWith(_, _) | EndsWith(_, _) => 10 - case Contains(_, _) | Like(_, _) => 5 + case StartsWith(_, _) | EndsWith(_, _) => 5 + case Contains(_, _) | Like(_, _) => 3 case And(left, right) => math.min(20, expectedFilterDivisor(left) * expectedFilterDivisor(right)) case Or(left, right) => val leftDivisor = expectedFilterDivisor(left) val rightDivisor = expectedFilterDivisor(right) math.max(2, (leftDivisor * rightDivisor) / (leftDivisor + rightDivisor)) - case Not(e) => math.max(2, expectedFilterDivisor(e) / 3) + case Not(e) => math.max(2, expectedFilterDivisor(e) / 5) case IsNull(_) => 3 case _ => 1 } From ba84de4e2143711733481feae6106127ca4218f0 Mon Sep 17 00:00:00 2001 From: snappy-sachin Date: Wed, 14 Dec 2016 10:32:41 +0530 Subject: [PATCH 042/142] [SNAP-1240] Snappy monitoring dashboard (#36) * UI HTML, CSS and resources changes * Adding new health status images * Adding SnappyData Logo. * Code changes for stting/updating Spark UI tabs list. * Adding icon images for Running, Stopped and Warning statuses. * 1. Adding New method for generating Spark UI page without page header text. 2. Updating CSS: Cluster Normal status text color is changed to match color of Normal health logo. * Suggestion: Rename Storage Tab to Spark Cache. * Resolving Precheckin failure due to scala style comments :snappy-spark:snappy-spark-core_2.11:scalaStyle SparkUI.scala message=Insert a space after the start of the comment line=75 column=4 UIUtils.scala message=Insert a space after the start of the comment line=267 column=4 Conflicts: core/src/main/scala/org/apache/spark/ui/SparkUI.scala core/src/main/scala/org/apache/spark/ui/UIUtils.scala core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala --- .../spark/ui/static/snappy-dashboard.css | 119 ++++++++++++++++++ .../snappydata/SnappyData-Logo-230X50.png | Bin 0 -> 5208 bytes .../snappydata/cluster-status-error-16x23.png | Bin 0 -> 597 bytes .../snappydata/cluster-status-error-62x90.png | Bin 0 -> 1639 bytes .../cluster-status-normal-16x23.png | Bin 0 -> 787 bytes .../cluster-status-normal-62x90.png | Bin 0 -> 4656 bytes .../cluster-status-warning-16x23.png | Bin 0 -> 611 bytes .../cluster-status-warning-62x90.png | Bin 0 -> 1749 bytes .../static/snappydata/error-status-20x19.png | Bin 0 -> 795 bytes .../static/snappydata/error-status-35x34.png | Bin 0 -> 1346 bytes .../static/snappydata/error-status-70x68.png | Bin 0 -> 4842 bytes .../static/snappydata/info-status-20x19.png | Bin 0 -> 847 bytes .../static/snappydata/info-status-35x34.png | Bin 0 -> 1419 bytes .../static/snappydata/info-status-70x68.png | Bin 0 -> 5154 bytes .../static/snappydata/normal-status-20x19.png | Bin 0 -> 612 bytes .../static/snappydata/normal-status-35x34.png | Bin 0 -> 1098 bytes .../static/snappydata/normal-status-70x68.png | Bin 0 -> 3892 bytes .../snappydata/running-status-icon-20x19.png | Bin 0 -> 830 bytes .../snappydata/running-status-icon-35x34.png | Bin 0 -> 1955 bytes .../snappydata/running-status-icon-70x68.png | Bin 0 -> 4683 bytes .../static/snappydata/severe-status-20x19.png | Bin 0 -> 616 bytes .../static/snappydata/severe-status-35x34.png | Bin 0 -> 1105 bytes .../static/snappydata/severe-status-70x68.png | Bin 0 -> 3883 bytes .../ui/static/snappydata/status-20x19.png | Bin 0 -> 866 bytes .../ui/static/snappydata/status-35x34.png | Bin 0 -> 2218 bytes .../ui/static/snappydata/status-70x68.png | Bin 0 -> 5709 bytes .../snappydata/stopped-status-icon-20x19.png | Bin 0 -> 796 bytes .../snappydata/stopped-status-icon-35x34.png | Bin 0 -> 1227 bytes .../snappydata/stopped-status-icon-70x68.png | Bin 0 -> 4790 bytes .../snappydata/warning-status-20x19.png | Bin 0 -> 805 bytes .../snappydata/warning-status-35x34.png | Bin 0 -> 1362 bytes .../snappydata/warning-status-70x68.png | Bin 0 -> 4938 bytes .../snappydata/warning-status-icon-20x19.png | Bin 0 -> 665 bytes .../snappydata/warning-status-icon-35x34.png | Bin 0 -> 930 bytes .../snappydata/warning-status-icon-70x68.png | Bin 0 -> 3533 bytes .../scala/org/apache/spark/ui/SparkUI.scala | 9 +- .../scala/org/apache/spark/ui/UIUtils.scala | 51 +++++++- .../scala/org/apache/spark/ui/WebUI.scala | 2 +- .../apache/spark/ui/storage/StoragePage.scala | 2 +- .../apache/spark/ui/storage/StorageTab.scala | 50 +++++++- 40 files changed, 224 insertions(+), 9 deletions(-) create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappy-dashboard.css create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/SnappyData-Logo-230X50.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/cluster-status-error-16x23.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/cluster-status-error-62x90.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/cluster-status-normal-16x23.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/cluster-status-normal-62x90.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/cluster-status-warning-16x23.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/cluster-status-warning-62x90.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/error-status-20x19.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/error-status-35x34.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/error-status-70x68.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/info-status-20x19.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/info-status-35x34.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/info-status-70x68.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/normal-status-20x19.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/normal-status-35x34.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/normal-status-70x68.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/running-status-icon-20x19.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/running-status-icon-35x34.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/running-status-icon-70x68.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/severe-status-20x19.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/severe-status-35x34.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/severe-status-70x68.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/status-20x19.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/status-35x34.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/status-70x68.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/stopped-status-icon-20x19.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/stopped-status-icon-35x34.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/stopped-status-icon-70x68.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/warning-status-20x19.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/warning-status-35x34.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/warning-status-70x68.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/warning-status-icon-20x19.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/warning-status-icon-35x34.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/warning-status-icon-70x68.png diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappy-dashboard.css b/core/src/main/resources/org/apache/spark/ui/static/snappy-dashboard.css new file mode 100644 index 0000000000000..1ddd1b5263e45 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/snappy-dashboard.css @@ -0,0 +1,119 @@ +/* + ========================================================================== + SnappyData Custom Styles + ========================================================================== +*/ + + +.keyStates{ + float: left; + padding: 5px; + margin: 5px 10px; + border: 1px solid #DCDCDC; + box-shadow: 0px 1px 10px rgba(0, 0, 0, 0.4); + background: white none repeat scroll 0% 0%; + overflow: visible; + width: 100%; + max-width: 223px; + min-height: 100px; +} + +.keyStatesLeft{ + float: left; + padding: 5px; + margin: 5px 10px; + border: 1px solid #DCDCDC; + box-shadow: 0px 1px 10px rgba(0, 0, 0, 0.4); + background: white none repeat scroll 0% 0%; + overflow: visible; + width: 100%; + max-width: 223px; + min-height: 100px; +} + +.keyStatesRight{ + float: left; + padding: 5px; + margin: 5px 10px; + border: 1px solid #DCDCDC; + box-shadow: 0px 1px 10px rgba(0, 0, 0, 0.4); + background: white none repeat scroll 0% 0%; + overflow: visible; + width: 100%; + max-width: 223px; + min-height: 100px; +} + +.keyStatsValue{ + padding-bottom: 10px; + font-weight: bolder; + vertical-align: middle; + text-align: center; + padding-top: 25px; + font-size: 24px; +} + +.keyStatesText{ + font-weight: bolder; + min-height: 25px; + text-align: center; + padding: 10px; +} + +.clusterHealthImageBox{ + float: left; + width: 94px; + border-right: thin inset; + height: 100px; +} + +.clusterHealthTextBox{ + text-align: center; + float: left; + width: 200px;} + +.statusTextNormal{ + color: #87B025; +} +.statusTextWarning{ + color: #FDB406; +} +.statusTextError{ + color: #FD063A; +} + +.divClass2{ +} +.div-width-100{ + width: 100px; +} +.div-width-200{ + width: 200px; +} +.div-width-300{ + width: 300px; +} + +.progressBar{ + height: 19px; + width: 100%; + border-radius: 5px; + border: thin solid #3EC0FF; + background: #A0DFFF none repeat scroll 0 0; +} +.completedProgress{ + float: left; + border-radius: inherit; + background: #3EC0FF none repeat scroll 0px 0px; +} +/* +.remainingProgress{ + float: left; + border-radius: inherit; + background: #A0DFFF none repeat scroll 0px 0px; +}*/ +.progressValue{ + float:right; + width:20%; + text-align:center; +} \ No newline at end of file diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/SnappyData-Logo-230X50.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/SnappyData-Logo-230X50.png new file mode 100644 index 0000000000000000000000000000000000000000..f4520e17965f3e22cb4c0abe60f20d6e24f17bac GIT binary patch literal 5208 zcmai2XIK+kv>l2N0RcfJND&YOLJ=f}9_cOg-VqQ9z1IY!NbjhCAf14M5u(BcX;Opq zCOshi(tDNia^KJQ=k0IKob#O@=bKr3_FijEoUXR&Evoxe007)lS5wj_&MU<61Vl=_ zsIgk~a*T+Ol#qHN@Y~lXN`!DU2A2jnmMR zzi~(K`AwxpP!XxWsD>dFW+&(AN6Eanxhh&xEa^X>A_@M1wq zT2=?i|GS=lMP*&_)Z1Buy_vzJn}?vxmMiob#bV}J+&TZ4eAMf_`tQ)d+Jb`Qrw9N5 z0c0Gnxtdy}OqxkvX3R`@ouc&oQP$4EW}I{*3zDhOYTK^YDN>VOZTR_}g&y}ltvX0( zk4bu|5mZ@Bvsf&AKF8zo)bco+1y1`a)5`xMiomT!0wAN=hx$VGb(p9Ne=UwL;}gDK zA6W1dd5fo=h0ll2fV@vP+A68<-hjlvrY8ktu;-zYu2jJ_u&pKAEpVW1er*#4pgZ}~ ziez#3q+qkL+zwy~83Rz_p!xF+Ua|J5o^w>+zl*TW&NaDE3W~xH`pdtKByxO*Y$o4j zDh~x^&URt$DAI+@!JI!n6DMK40&HrGImrN0;)CH}+kCu6pE`oT?fsY8)Iimfpp=)$ zw+ey3bt{@nWBisab}mW^)E+j4-{cHDEJb&B$htYZ9Ah`z+uIp=wU^*smvf2z2de{% zi;D;ijVu0)W4b~w(WdzAA2;%4g1+(w4`4SfXr7DE zkO8`PqHIkots5H}8dmyp%d92&`JGPk2!7Rm+Wp!$_imcoWZ9%PdoOFmDLoPQbl26@ zMFa8X<+(&?c+;;{$=UHL56u69hSF zcz&ga=>)}5Yf({=WVnx7I=A}p?CjO)34WR}6k8Vpx%QJJ2LO=>S2RUBaw?Ej z%ZU?(bCz?yrM)?(tS4IU$=RL*fkV0#1?oo%*|zr%*(5uH3us#w78b(701Bi@iwF(H zd=iWJWIDe=Swr9+X~=J!J^rl^e;)>oM$e8mgwtzqr(eXJna^38nXNq|*kJT=@n1b& zx?qE@F3ys2VE4h`?-K#hw?X74&R-(sc#;^N|5Lo!~_x0aWe69Iz0THO$@)RX&UkHdV>(Sy8O&oNLaI+m@s zHmA$UfGX?lKjM3*aTp3J)O?g9;bb?hw6q_`$V&?R@Lg-I8PVmltVP-$z?4)~SC^Ly z9Y4g-u}bLb2!W%8Y;ZDMb_vP+633Uge|%f2c$Jlpj*eX8Fc?f64yRD43nS6GRgo!V z!gnXbYECj7!p|RMU{GwGj{e~{SA4f8SzfSge>~@L-jlEvj&{yyMNLBOx5$0RKQnbc z`k&qotcuatySkzm{w^AijR9d77Z-&3W`Xl%CX9ZZt!WCo?Dvx-e7)yhQT|+@iOG0? zAi^NW4+<9+rj_KRX)AF4_3PI+_O@#RRPBvJ=qpa{%7_QrLL|%QmTbw;a+-{ zs1wTzBGr5G9gnoIm=5^-KKmZX_NZc=SS7>5`gNAHoKPrKCNtg6V@HJ8JyseKT3Xoh z@+VNJ;oP1650YZ2W7isPfsOODhpKAvIwyaYMELV$arvlRhyKQbQPS!Dp|y3MDdzLJ^eEG@2YHXn^!|g3HQUr^{%vQ zXZ83}CrVVz+DO?qp2EVyG`%!RI(BYQ!~n0bRYUEP)wo%OnxUcLOiQgw8iFZP*la

4!BciR`lG+zd`#o$Z|*<3&EJ{qG(NkdV{2Mp7=0@>2s@(kDVpcQur) zt=W?oxz%Z}k+sdQ5`qdJgr0_TfJPfyTNP;VuN7>O$bWym;ho+AUsUUUGDO}~0Qo#X zo_mgsjS&+z(qiH-VD;^#R?cCbhLnRGn4=i|=O;;a5dbI@diG?12odYB+wVV$90WGR3MG}I7fX{|cNnYlbMs6nwqwD zKL(MhTd2epH-jT<^SYdVi9~|g8Y!_!GGVXL~yr2J()9d1Nni zadJpg!`Z@pUw$^SyE}6eGOWsd67HY24I_}fq~J5s_&zk-+r-S;K;yDMNu)+`WGccU zXz@u`M{8@uEh_%_V0@Zveta+igJ9z2Z+>)8e(^<# zK0&6=e1bdirj$(Z{QMAgO7P!3=`(Ij6W%FlgopdnJ@JZ5rKiVc)k>emL7A4 z629)95tpqWk4^1=Y)o5Wk8!oNr~dtJAWp53NQxha+XNQi&)Qnt(;K-`8B}c|_e(U@ z)IjD9;IoK3%13L2vNPv#dUZ`r8Y+FpwS(lYBIN{h=Xn9UEjU`du8OT!`m=d;Ioqhb zllj=qPai)%Z#HN9F{S$)1R#;fi6%Gc^39En1fMc5gH=2P=JKKvs74 z`Pq5l>l97l1Om%n6hA`rP~UMNrJl|owiO(BND7Z81Jtr>U_p4jDb}zR@L+{{?rrlwI7o09XtS;@$+`f(-UtkVX?JS&#Ea8n5eQA!)b|ezsv7m6Y1}6#=Q{ncm@$}!#_xV4mHmS< z>NBN++Kx5^W1}ouR)n3PdbWF46|0P>6LN z`qt>^Xt(Kq)&`GicSVa5>gs~zI_=OCnly&D%`8%@ggH5!+t`gbKBe7O$##= zU$X8UGK&X@8#{+kBh++sDoa}+fVV@wR)0~NSvn~1-^Kan$&{K@QKFc=?RHGQetWjvbh!K`+7IyCC^T+wX1 zv3u3R@(~#zeD=j%hK~zlWBCl%>4)|x)}Euy(fg9_yUaocGzrx9w(Vh#*|i^8h2~Xt z((ePnz>CU=P&gF_he%%DTZj4O|01e3Co5|=zf4E(?45v#ukJmdDqwEb?Ay!2#(DgM zSM3(DdfItXJZWn5PdC=rVWD+;mj#i1THP8e{{9$AHEU_tOUR-p_4GzuX2ygp-)md2)viyw(Jh$eW^R+T( z8&TrY(iXBFvC$v})5D%O{(K=*P%jzw$5K+}1Uq{2mf+Q!H}|&61cR6RBM_MgDh?o= zIVL&z;_dRrf`IDM+b7%yrn_N7_Ha)*Gy#3RiOn1UQ^-_?+thosrTa|Y{?U?O6CqhK#oq+Lu zsHDQ@ijaAQsh-TolRWW@fr!XQX`{TnJX%K5!+0_+wJj#{$Ggl z5yFG=Kc$@`Cj}MCaez`Lyq9*5mfF2lTYBy9_bPT!?l@*7OP~T{;rll~g`HvEdarc&%mdzWhN{OtJ3X|NJPXbSGo#APod%LezF%JeTL~o2|R+_pak$iWk?CtHLd=`41{t=?Q68INEeWx=bhE z+T1w)iw-vruU1YulS{zXRMhr~?w#uAR}mRwc3dO)fY2`77MaFI=v`5W@MIrtNLUJBr72kJK6JN}{8aZiCxI-aOJCUNyW4 zq^5g^Zcd24&EDx+smWt{rdsc3gKsiFqXIkw0*K$MYDWSUAm?CY9HK=3+6!(FnMyw7HS%6WKr9PSMkDsrpafmQTIOVr<(2d|D{ zEgUI&+r2(;#FiKc=^Gd4j(`1{S^;5*^PR{bI=MW!&;9rHL`&YA|Loes>-@JKhjVp* zU4H-j_u07dWTMN&;c!H&gTsm1-PIa-fZh#14-)TzVZSrRDOD&c{_b_KKQS25*P|)X z<$F1hjV)GhYHHG=yj9-H8wCIl3vWZ@`#-`%n5d#Y$9{V6B(0LtxEZs5VZMp)j>)nIW!(Xi?-7BKtqUi-n({#st|@Vis#VF`V}pJ9VTpO^D*Fx?EaCa*Cxufa7P~beYJWS9K$ziXh$}8$>}3`^VkYrR zBWkoQaGNqU)BEgf$h&SGQn=HvM}a6*3K*^+oH_pc_iy$4uhusIqP)e zSad^gZEa<4bO1wgWnpw>WFU8GbZ8()Nlj2!fese{00F&8L_t(I%bk+3Ym`9{#echN zCMhCVdWgH0Mm z1d|X!P_pNKYvF#fcYGJ_ZkpMBZ{G}Wc3_BzqU}MP8A99wHh{grmJ7az-_;<%1y2;c z22261OkH)s->dmS-Rk@*il%%%97R(UU5uhPhVlC>idJ=jjJ?S@MbX(EEdWammKF-= zTgltv;h`ux(IKk8O8KpPGcLEO+vwN#Of{HF;9}!tH-E*cWal z!=|lRFMn)@U7O2!rnHcNx4=n8i(@8FbQK7Ij390Uvlcj)0U5FTZ=hv?GmIDKfrl38 zFn+wYz{@ekYcBZb7We>oAdPAvieiC3F8DUld~@Pk1qKCdw>g`U%O8&Om3U& jz@-MU2VS_~Ze zSad^gZEa<4bO1wgWnpw>WFU8GbZ8()Nlj2!fese{00qxUL_t(|+U;CTY}-^E|J&|D z5t}9o5*!fX1`@}7V1_y2w`0q~&cp)hiL z4bn@1g+iQwpo$&}qrfrV4me(%c5N-f%XUpQ>O>$IHxo{!gU7g=tdm{_J@n3vRwoAP zu|1{gM!@ar+i5f0PV?_rX&G83~>3mquW+v zvcg8!^NPi&w}G!Dwvhg#wQ9exJl|>-8gFaE&1ZS8g4JxW0hctvQCRwT?(!7?Z`G~J(0N$ zE#v)~nCBZ2ZaF)0kZj}J1g;hvu0}+!n>BU2@%Xv|q@={5I9VQ?jnnD`Xv{?{K=|BU zgvwOCTb;?Ou$QE?gYHLErV7b|>^-;;HwdA ztf%8ekSmPoT+ies)5C}jrUvX&Ejv2p}`F;ujU|yLw4}Tbw zizYBpWxD!I;ymvTP_C%+4;wnf*j&S;pdMJ!$P2SCM z(%DI6&!!3TYwB(&unKa!L1727R|na1BjSxD+9|LK61CQ&9HBZ5boCa6Tjih6zlgPI zVmg_e6CA0m9txwZ_!|NMU`kH|06-zK30h=S$8GN)roPhw08Zy-b`zaUq_0j7mbg5T zy>_^9m&(6H(9&n>w@hSrfqq>&%b~;aUYU?D< zG*l0Tk}B(>E^DUs@dzL?uW{?`oHf8H!sMb}R)-Dciu9QoGjOH^|} z`=ERz4|19J9P7>gTVIU7DlP6DD<3&xoD?AhiSlllmx(EGvN9TiAr>fRa z|G9I=Nt70TM*skGwOP14UfTwr3E3&VTb?`*eiYIe?{>zv_qBHPP#DQ;lINUUzkXy7 zcxq!74qd-GYBne{ll3p->j`_6X0%Cv0=JW~`cX=yYrX&+S7`HG-LBC%ELoY89G`@M zN}zzVrP!WE?*WhGCVUNAUW-GCEt}Wl!)(J&?AwShFsj~8YG9ok5d<~2tj=Z>{(r`U zz&6#W6TY@_J2Hdq-2Cd;tkj=Vg-GiIW(hY2fH3$ zTs;#Aw#BLtoLt>Z&0AF6&z_xX)CsA?n`dqbmGw3FEUTCU-(X6s~0wzO)@@ z`-2vX>C-?ZSBC1$?84(eIB!Hak!ytq?t3Kp;Pqj2WR9GhmAnQgwx~v(c&hN%JzfLM zs&{tnjC=kz@jfOWe2pHS*KY#HyG4Qdmb>n$!Vz^4mKea+Q(yyh!WP@)yr lw7_V)aV-Kl8qasm{{@dD(?G5%!Ycp(002ovPDHLkV1l690n7jZ literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/cluster-status-normal-16x23.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/cluster-status-normal-16x23.png new file mode 100644 index 0000000000000000000000000000000000000000..f3bacd7ae3f93aff710baa1dcb98ed030a733abf GIT binary patch literal 787 zcmV+u1MK{XP)e zSad^gZEa<4bO1wgWnpw>WFU8GbZ8()Nlj2!fese{00MhSL_t(I%axN~XpM0g$3Ne9 z=ZqP1)jSfTg&n$fWxm>{u%JO?kfN_>CfkohWea;lqAU`B$U{PD0bT~HNKh| zu%|yuzGaBWy93BNF+KwHx8_HLqVXVkhuQTROK8jQ)?)B0Et-@u~7VSs`5@qy_M894RF)tiQ>$X)L8B5QVQy+h%$9Y zK5Q;!R-9v&G?u31ruHD{LMi~%pfjL~J`3phtQ^8=br?LY+DqEYY@lE*=#oKDv+Am z@kGgKjQaApyGkXPMloH}kJJ?t-^;$MfV!yw8S5%54Q#0Z3Sjj%dn_yC&EN0R7J(L( zamDOF%klsvbASen^`NG|12muVnu0x3kZuW(mI4dGy1;6v+JcAJcynOb#_~_9`e44I zq!aWq6`+?kkyh2#fRQ#O4+1kAh0^K%mr28YVSXgYx%d;}@_$But0}U3VwH1Yd=hk` znoxsLWE*<2zR*74N@o}OkerK&VzxPOP%d?{ezQ%zX1A;^GY+r RIyC?Q002ovPDHLkV1l01W>f$G literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/cluster-status-normal-62x90.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/cluster-status-normal-62x90.png new file mode 100644 index 0000000000000000000000000000000000000000..f5a95585a021317e2294308a282bace687d1fd5c GIT binary patch literal 4656 zcmV-063^|4P)e zSad^gZEa<4bO1wgWnpw>WFU8GbZ8()Nlj2!fese{01=o;L_t(|+T~n%n4ML5|Gn=y z-@SKcGGrl03|6dAux=n$P%#*iS?)G-XURrlv~{Vdv=3Tp75`8`v07`b_|!*v)EY<- zlUeUNGf7yY7R4@V)q;vbKm-D0o4IrE_nr6cA9u1$X3b>M%%ta^dG7u0`Of*BcmJJp z2AG4bC7rzCbdSMv32;+7MF0?3dG+ZY%g!a>>82iSPB;IWwfx-wD~ZSZ;(rTUyhzEs z830n*y3;lm%^ADSC6_9Ky$l->5aDxl20V1+vl@p`sfyUpHS=24qxowA0Bh6rp~it| zc#PVGA|EBV>Z6~d>#la=$O@!3e?yLR97bp zNjq zY3pn3hIGqX_52^J@jnK5@RYlEv~)btnr<2O`x3eTV3p$}&Tz)T`u34>Fy!V1@U^*C z;8I}u;+JhX6dJx)Am>@NQ1}S>=92)>)zm$D5>)jQ*3tN^cqn$Z(Hi)f~@ zU`p>(_CY>DE#QHT8N{QbHN)1yxy=C3mW|I1;8*N@g&nRA{zVyYCI^I_Mac{weQv-* zwLwJ2O;>xH(3{BKC2BK70Es~$et*Z>?1Swc2>`%XZ~L+*RDJ^oa#L$Ac6(hixFB6pAF@RxI| z`$l=YnxMS;Se~s5JUvi-MuBgS_v&IgrqjR!%qx#+56<(+E<2+{L@Lu#6HswA>EvsuEH58A|Jg8rV8ZwxD*yoQ zOx-PGt0_9_DTz=r27_l*;w3vABO&M%bsi}%wMCCwcCI^RA*te!E7RdK3OwTYMr8x& z6!9Hc>LTG}cineqRzHjRHh~;SprA*(li4TFC~)B*P^D><_6q09m&|mWoE@TPhA&nGpeaPI1b9NXoXU7k-Hl3|&eRwB61yAE9PnZ54k_G00`Jo) zl%18}z)0-6W=1L3>TDerXyhB0dizqXnG~|C`u^{zr)d6K6k_*{&bkVT2UTT-DugCk zNFfCiz>c0SdZpKsGyC$ffiuSXOeyiUd=pZAv6sj4tKM`X=#i50w??LZ7b^iuaHNab z*>~k*187gT%{}04g-VWWgDOLga}ihbiz5`$2_!;?gFQGc)lq;zk&v1%wD-1N+0oQF ztx2|ITSonzjW+3gQ@dgdykhtk-xN0z;g7qceZwaL0RgS~xQZV6Vj51WQhh#QH2j** zc=zD+z_+$`C%U@gg?i+o1E){oiOeQMhUG0j2Okj%esC}}T%K^*ME~ogG7(`?!i9kr zuLgkDd~C`-MSkJ2M4{npx#$4eva3!H@RoclWs_SmOy*mJ0EDW&yH`CR6UTDBWxCP^ z0Ad1E0p`B;d@R)!%TMW4`(h9L#3-+B%dWn?GqD@VLTvVdcgMQ4CExM|q2K~XRzu>Z zvX_Sd9GLE)WeQ-$fR#{!?r6)$-URAcSue3V%FVh~1Hi=Fyso%^vX9 z%$gDVE&q*@Ibg*d4z&7($zIYOEBwKV_dPKHO(Q~DAvh8!6a+pD;eXncYro>&c=v>T z3Jv#zi^~U`@%I|x8(Oll&vYg8Xv){k4)Ctz4z%}PcST?vJf(#~sCDF5O=*MGhd*U~ zZ^~k2oT_V)uv~e74tD}TdrxA*-*&ZqwX}G+_R~TN5Z~OCUwvmfwi7M+#O$cLc!QT1 z$8c~JdxX1aWB+v%I>O{YJ#wpd|2VMnbJdfafH(*UM$Lt(O#CYysVv(1W8*I0J9=BO zCDr+LQv8U7pvK$SmThY2j%CqSh@Lud3Q9}GBSeBQ%H+e4tR3?A*GyPP*N=o3$G0d4 zC&$VXfmAD@(CCNS`eR+44f%1G@9yTVk#HFNbvZozGaH1e9S41D`l7Gyj22EExHDcg z`HEu5mrxLbf^dkQ3jx5!%*OGF^Jzow4+4gePZk#CfHrim%b>Zp zdFH@}!UqnK5RRvld@Vh{x8}dM06?OUn$U&q=hBau2ntq$KQoU@{1R5ZHPP3Y3>J(Z zP>SCEgepn|Dg~r4Mm~|uCU^8U_s$%6q5k_1TVlv@Nr^|Eh`6?7p?Oy}+M}(R)R=zp zLL&wm2Yw?(+DN3NGp%}+03y!_-MRa!htZl%j9KV>-9vkcGz_&NhJ&hsCBLC9+xEJ3 z>dq%HjQFck+0@@(ru(QVmmD>rT(nXnYi+3-j!)sZg+LXSl$NFP z@!xkPvN}O1{9YyVXOJoaum#HBE<3ky#=wp9wg|fkRI&Vb|85uUOvf|YTo}zOG!4{Y zIP}7|%3iQ%Hg+pSsJgkY{@d^Fy?)H=jPG{PC^nKqfeqdu)_ibAz}tG7(9xXkHqJh( zq+^P>fYdjRR;BXst>l+q0syhx+W`RS`kkmfw46h}c~SJ#kJwm}g{q-y&XROLic!Bcin$VRf>@2<&S zaehAjz_I?xOh82cN#@jnJ0=(xz^^Vn{GqMyU;Lfab^t&}+W=I^9$y}7J62{h0jt7L<;b~w{*$SE(?tM~9ErW8fI}dsOc*$az*>X@d+y^) zo}4zjY7GB*Y0=(ACk;F~P>-E&`?u%8^79iVek|coC>)8B1>CMgK6Yznq{Q?M-|x9u zI88wgPwKpB`34O_u=wnRz}My)msH4r__%4P3z_wkx#vnXrVzzBVbxEp z$);9RYV^`Y0y_)-?U*hz%hH39!HcIAr`4cDrRyhrUS?h2nu~p;D>in&xp|-&>D4{I zARs+;(jkk54Et2d0f6VQTxR>ch4SH$GYr28TzEzdV!UAEPaHNV!(1Mn1>lYo&GAlBxxP@Jx8tS>LNf;^n!7v&NsDgmfutd4Z`9qmG{JRv42!mwDeWVyw1i9_C|_7 zAxM$I@O0`8R-)nujp|ha=`2ee5GX{0O8`JEm8Q{aNg~tC<2ttshs~gI=28`NIHL5C zmcG~nRYQx;0f6gvufsztcl%*mbF+#<#7$XumKoL{w18*%q%47<>cJ4d6#$%SH6T;; z(LV{9q0qR%v$32OYX_hAn72A8pmfb(xY61eCyC$8wV^iTeYTMq5t`$Ez z45rVBn58(wO-XB#x!6q>7SPE>uB=0T*y)78*L1JL_O@*WM~;=8g@DSoe8cem;a7I9 z?HZ{~@dVGd-Oi+JRdykzhBEmT{hyTDqD5ZWcoPEO(Nay#%)h`7AV)I62 zsfEzIflZ*tnLQr>;<-jd)U5C!uR1F6`krf#c+7VluOqAs(Lo+rk-FoLY0e9ILxx3% zv~7C|xp*JSjJp8+$|%vT+lNWD$O7?i`;R+Sb!k81yZpJktQ=8}1Y z{p&sP{QCi*b!*GTW}*94K>k0Bvhmv5rR|{+&;TIh;3zfTbeU8iSJ^(23 zfqG)S%mfAjmmWN->TTJ^B@W)tl*`NMs2vlG5Pa*v^T?qciQftx>WSxT8au8sk?N&S zLuCmuKPi6s&$yxnNf0lHD9dslRdy;I>%0SfWbOv`y#QcqE_fIkoX5(>sT&NJZIJXv~3T#e769ibp%+>3iC;ZP!JDL$4FI;t0y5&unkc587j#w7DLn5GjvN zb>?RyF#T*@I{K$YQ`|;kK%!7mL{zq)kg~d^P&pewS0s+o-9^SWR#h6Yc%y^^sQ&&@ z;PG9nUxu@*DZ#EWYCTdNm9F;1dVZS7H&qN;)V^+19GIVWJ*w(7u$zKV027!ZpdLVQ zOrNYUPHAv#pb>?-zC0^K3}d1@CG$??Dlh>7O69-?qLr%dn@vZKs^;q}Tf1d1D0@DI z#8e!?S<}cZG|rZory+*ZM6OVm=Jz zN9WN0%Q$5n_iY&f!4vyA0O+kxV~na78Z*8l+ya%eMaL0wRe;JpUGYq|H8=VU+)>~y z`8d+C`~#}^;n||*fdr+9t-TWfy5gDffp^DpXwJsSGv=+zv|r|V>Z(v-jof^1 zwn@&hPksw2+tAMJea?r!+^DIV>t;?q`fb_IrInK_s@p-dYwZO7nU!PXFt}v zI(t$Anf8IKXE9VxC-NV7&OCK4DGey-X(fPwAtgF(nXW>i1SnU|)K*S(n->F??^l_c ztJBfG50@}pnu#Nvc02wyakNmkC@kJrl{h6BeuOo0Ih_Vga0FN#Ik0DfpGHbmx3T9Q z+#c_J768^1TCgM9jnfMJ_|}w*QaWBx0BGuO_!I|pIf$=>k`VNQky{p^qhV$=rA3Mp zu4>V7Jrp2%3aV^~9t&U|AR7K1%Z5rNG&|OSIUmk m@$554!)u`)J<-{@bN;_mef@U*M61mJ0000e zSad^gZEa<4bO1wgWnpw>WFU8GbZ8()Nlj2!fese{00GNML_t(I%bk+Fixfc+#edVS zH7i`Cm>hluf*9ywAc7*8C?YKe3W6XAf?%TTKOiPzU~Hg)i3aADavW-~!5Nr{;$d=$ z9|+~1AZSM__ZtLniR5mXk|s}*B_w*fzat-vqq{7~JD0(x53 z!0SP!4YX6~nsuI_vY{FPeUsw(P>Fq&xu9}$oPM7Nm8vGlXAo3kP}!Bg-_*cy zX7pFq`BPcyd*cj(%ATOIc?A4Ezgy=Epf^OY`m9AzIRtd9^F82URzCpj$OwZ zlXc!lxY$7Q&N{ygd;^w-oVcnoBe0Tk|A1Mb+dy&|xSQs3)jI#k^t60Zf<(BJ9+kk4 zb!_rD+rn)C(~Yfmgl+*}^NnrbwcY95IcOod?k#Yb$>Lyx6%5d~&L0C7#sJkKC#~}z xt@8sPV4dfWwvLokH-JkG;6>n-b^cbK{10C2rM!(YI0XOz002ovPDHLkV1ld!2b}-_ literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/cluster-status-warning-62x90.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/cluster-status-warning-62x90.png new file mode 100644 index 0000000000000000000000000000000000000000..65e0522077a1549ab2c97e1f625445d0edca44b9 GIT binary patch literal 1749 zcmV;`1}gc9P)e zSad^gZEa<4bO1wgWnpw>WFU8GbZ8()Nlj2!fese{00uotL_t(|+T~kYj1yHD{$^$= ztU%C(L=$-+@}N-?l}Jcbgk9S0OnZ8n?skD)7o$Yuli|T9jlS?^VkD>`V$c{|k;`sx z)6TTr?GkSxA(EItd@!0AeE^NC2FOB}+3}%tZOcqMZD+dEng3;zozrvXoA3KC=RY%m z1x*%QirH(>RstMIg(x`HWWl94j`3Q+p|snywJ0n5HA9M&BN$gRzsNd|aWh$`tqi*B zgBvUMcQj&KO6rw>YvtSMVVEIzQy3>ncgORxC4dXTX>6N#tVb(F@u{%=fJZYt@Gx+G z-A@ZzV`720?)em81^|NTL2Cm)u_&IC-_@4(fs>=vZXuDnKHNi!ZU9L7bB4N&VZeDQ z09kM;E3mk;-nirYzNmH!R%mIxNZMx&;9Gy&)YIy3!^wau>kKJULz8|0z|liTR#XPe zDSyTq=}fN#>&d!yj>@q{*YdOxfQ|4w@3gN9h?yG{Lx4APgNjm$&}x=-XFx<23@K9o zD7_nob9<~^xi!mnt4vx>>P-i-=@I)@ZuKngK5fW6?ZyDhHzLdmc62BG+sKFDFL=94 z>a9Mk-FF&~Z@QkMc_D=KK(=#HofZ2bakGX1A~ni=1wiWY|0+=>c%Vot~4{0@M~U+MULL8eOM`O8e!3IjMQD!;)5!T{H$4w#o%=8l3SWw>ztw^#fs>fR-B(Z&4Q(Q$9fnO4Ua>fxH{++AWOHl7A`r zs-aC2)yX1>fG)|(f=h|ro>l;W%ao4HQl4o@*DlFM`}@cACo%x+O+-tD&XlLEPLCmZ zxu4qdz*qi@SIv|=0KE3^i+vxCee}29HC~u|h5CH|2hB@e27c9ZL$k`IE9ILYA_ueV z$huGGni>CD-!Y|WA18SaT%@?ESziz@_^bwNly}yEU_Hx<^Ee+jK9v0FFwAye{JQYilQK#Ep79r zyaJwC^Q;S9Y-&W%0Pw?&*=v>mTuiT2CKn0{0PHJ_;QX$_0&HetQqr5bl$&tDjMsWh^?n~U&`ad@e!;KT-&Phn}?>NG1yY-0B|;3 z|G=J;2J8vEp7njVjuF{{EdYlERzL7R=jLzQh3N#5APX*qhC|I!zyhc$;6~XwuxZKPF9w_m8~|u8@3m|T*$fNZc~#ZcQN1qX zP6}w1%64|iI(}-uBiIh+1J0=2v}9f-bvs9Ph7>72L)0s`u;{Zj_?S;n2Oj4#w%w>i zr3f^4XLj{4Ed7kCh#*){HVVXp~o(#Hs1dT^~aTN3}m{3o0h$S?N=F6 zq@G*x=z6OG_Vw;)IvBV7HSu!Gzs&Hk6vkOL4pV{G2EEF!|2heFd}0hI#wOaDtL=PE ra)S^CbEc`F(sx6W2MN}gZ_fV%l_ubB=3V$y00000NkvXXu0mjf|LaBO literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/error-status-20x19.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/error-status-20x19.png new file mode 100644 index 0000000000000000000000000000000000000000..68ba8591f824be911b11b534bd7806eb1f4aa8c9 GIT binary patch literal 795 zcmV+$1LXXPP)i7BQVSrvp#R4X+=xw1XZ$xmPP zp%+6IT9__I3CBFUaBMs5Bx|8}PZEF|{XR~j7FUJlMds`3 zjI5k4YG?q8B^`((Tb>sB&o5x#)&x5?7y=uO5rN5q&HVVKhx@Vt=2(Z*Hty->`oJKB zzFrqwQJ`h9ur4F9S}O2kN^b#}(Mh{qV&d+faxqf-!1bdpah(7ZN=nbJ#AaiRY^w)T zuCR2`JX@-vEOvtfkIuMwh3Ll#@O_FkCbTT82OC}H#2=kd4qcw5RGHuACgDntTQi?k z^8-yARh_N}X8iK+xk)&8zr;jAx%|vWwLC&?YYj;!N*GuQM&W%uPzsb;AKrenjCqOx zuiuB9zFopimfMTd-AOQR{D4<+6%LJr1YvFB$Y;yYh-GqG@q^gSd26!67YE}6zWUrJ zof6WidB&NNvOF!Ey;q8m8R5>|!C*=NcDI?1z>kBqS)P2>}TYL!vwzqo|Oe zJZOxGiNr!u5($Y|1xYk$VNq)$X@H28CTQgvq1<-a?RK}jb2&b=v}||TIj!(L&YYR^ zJKy*JE;B+($yobsWeer3E!jB(mI9iZK?o1k;);=c{!mD5(JRR!C*dotrx(hco5Cx}hAFYjWe|7Pu zng+`j>bRadj06&Cxx%r{+7Xy!M!qZ+!n3}`DDued_R`Y^58Nm4#}Tf9oQ(PbO8_*- zHXlq6%GqHL)lC+xY%br*Dl6c_wH7s%0?$_$b0#N;U|f&+p2A6%%-7Kr5~j9Fu-&nk zHdS5F**uK^THcp)RcgvJ^K@m`S5FdbeZ^94BZDq;hhRKv!xrlr%&aTcN+K=Uyw|1e zNU>^X)(O_P7)svpd>M6Y)6#sO&?8$$F-R&Sd25f$v44C{o-4iM@`VNwQz+?1N}?K& z4tJv@Kj!c|4tZR>F$B@u7qGvpSg{orG|f`0%sURbNlgg)R2HAPGJ5)}Ny*PWg`4j@ z?6RvwxJuQe680~EW*VGUS^dZPQJ52Ff|BBFKkD(>4~0M~D7rSppcxun!-I5LJKXJ$ zh%uhNbDDgA!sFD%QSb}b!xd~9;#UA|_b*;n6U0LO7?&J5<&R8l$r)zR z!#Yzd1zYyIDiyf5rixu92lAFi{U_jA)w2KSOpWu`B>jV<@0eaANTi2qlfr<$^v9F_ z!ICp@)Do+2<-Cghc`+ye0l1uyN+7^{w}X`YRI!`%1%5y$5ar%kd#u}E75b^tane7o z36=rzdj(Pz5o=3+7MHrXdBzhlP4E=ZdzV2r#k&1f2ab7JWf;A3GFxUf_Zk|H1KaNc zL@%&3*6o+g*gLjV&5E#P&Z6`?Q|7z|JO=DhX&Ym#2iRa}JQC|J6zhb3i2wwHS#sB} z@00006VoOIv0RI60 z0RN!9r;`8x010qNS#tmY3ljhU3ljkVnw%H_000McNliru;0yv1A{ZVOJ-`3}5_d^N zK~#9!&75hBU1xd6f6sf)-DY3BOzb#bve=33NrD3*X+j8?B^094po*$Um5^E$fl#4T zR9(~$mHM%&P@&}uO24$UrG=IXoCJtd5|hS>;}FM7?8NKXW6yZ@yPWNP`r$oi?%eTs zW_<5W$s=9eJ9q9q=e+;-|Lo7XLL2136ORJ$;J-h6-OqgWQ@&RT=m*xRvQ}XY&;#Uw zE?^LdfDvFCP=Q%sRD_HF@N|56rSoU9+;yy0Hl*;NgYVBz&KC_6sQ^_6Tq17G3ptnfk2g-2o-_qx1Ne?&1}?4;?rjN@t+ms zzg|w(sY`czpTTXrUiK4O*o6U%U%AVCaK0#U3DEV6koUm%h<%SY1^&kvIp`UV zsBTf^{R)p+tGjCv{k4dmFh=4;8`L9vHp^^-$jUos6=DlXs05Kao-yL*eYSNJSX1!z z{hpCiR(U}aegK?lD`V!FVy#2?`_Ik)@5`U?yQ?9$CYA@R)khM`M`KHWC4eAO!q{EB zELf~a`;4M$+$3sYx(XU zelm9msJ70SPe0PYlt2GW&P1vo`R&(O1vJNJeC#>>&wTMs5)wpxEs-|31 z5_JsuRe{2S4H0py4MJtQqD)uV8YO(*Dqm3LzPB&y;9qm703t|UKV&`p%w1Xv9p%%^CdOQQAyi-qY!3M<^ zrYlMzCxhK3o)FP~W8`WE%!)Q(1nB9|D{d_IrLo-#jhLQTQt?z8Jpkxe5 ztg{V9ppbKbWcDK4`Bs3>j+OVJvnZ7EoXq=tyOft_Ji}Yx+!vPtzMFpjvnzR}uRZQ} z9D3Vsj}v{k9B_XSb9)#mv2E1pX5#||DnUK?8|)Ugt}_gFyP$BY43{P?bG614p;-@> zq+Tnra#L>ypG_=-s(c5RYc&AU0L)^mJY8YHDi4G)9|&T$1)*DjwOMH1JX5k5r?&%& zc|&&*HVghhS@44F^FhaEtGpft2vKXzWfP)va2XlfBhd( zvRHGLWK1>GA#3??81qOFYIl^RHpR3?>mY`?s?ysjY+G;GwMn@1X2Zq-p{pd+Lgm6$ zC5V)~UsowlS_da%&Rk8sD3LAQCHBS%I$PEKz{E9-Iq<}z^Njh+-^%&rny$Gxp?3u_ zcZM;&ahm6}YEz;F3O@9&G2F67xM!-K?)%&A__r{6# zEHt()F(n5Z`h{J$7~Xfg$Bqqx?=8Kn-6f&BB=~0O@!dM9-?E)SQVje69^j?z^{OND{`L8{m+hmp7f|$)Ink_}LVGOzFeF2TAU80>N}P87G7J*p5VkXSmSnBiKe_f$jeU08SWQ}^7RSngKk z?jY1sZ0ku`GjU3sGG(w^*gNcT->ruAy{kW1Det`2WEoChwp7Ekv?=XlO&a`_f|%P9 z<*aY$yM{59Ks#!o?n*2>txCaPL;mC&^JDx>p1qeMOgq09o3NSQNnh0ZI7ndV$)cqqX4@%8}{64 z80x#eO}UuJ3 zL1{fd=`IO7HyL(qGW2%b_%Vz)BfMp;uxpbLCx#1GEwfdpQl?EsFs*keN~&xv2W+iH zi~<#N>hFraDt%$BeNn6}0V23EV}GZxV}o$}O+qQ({Dlv83m?13X~?(id;pjYzZRWS`D-a;D6_5@j4L(;)G7C%^BS+bkBRx-z97q5Qh4s<%qSAgN*}1 zVh!iV6%osoX{8#rJ+vuUWkV2glL!R>GgDTi+`@ z{$7vA@AKF=APs8J4x6KpixLKcn86?-2jB-0Lm~_SezPt*>l+nwuz9Vpd96^)wQ)OB z=73>jLW!+qbW$k?uGZ29dy})4!8l<+#MxaF#kx+FK3eEXW~$^CVAwcdDCAnx`=zA6 zQ~1zsj}Pwj=qw7gi1`{(i(;(Ro;YD3ic`kKmR_I}@LDxug`BXa8`k!$E+YM-E3*~ zMt8Cd2Tml6O)Gt!!o~rYKCI;8*YyhT+3HSm@{%$+r!A|LRS?A!aRl+LC9jH4le04O zpcu&cuI93m)FK$4QI3r$ubr`s%_xK2f?~L1!^+?9DZyQv47CU@P9%)axP_VKjc?W@ zHwzgfs?CFZcEs}98Oy~3|>qx5&y(}G$A7blg`No_E; ziw$2E*hLAvHEOAauE#XB*|7T-vt+R=o6&2?@^5e*0S9}F@N$K{+T$@QFR8mX*Dte)j&BrrkovD!uov9{4_&#mAOsUPey!C0swQzmupF4nFOx30*q(eqQ_^P+Z~^6 zTxb@A=`xd>i;rh5%vND++8OMz8K;W#tBG7XFkN;5VhgR7zzFAZKId}2BS?0riojWc zb3o9nok=WILKoSd9kX1T)cJmqZ|ai^tKC&&oZX2M7@0`>+6#cG1nyiDb4rxdC%c*g z#&Eir|9T*)KKZ5mNvj;w zRm`WY#(jGRCgx!O88uC`J;TAAN4bIhPo^)rK37$%j0hYD&S(C-6)#l{mA5Zjj-0a`yI`4^ zbJJ7XHwe8Q^~iXsh}CM=#EwFJ`d}e*vZi}6 z2V2&;{?yHbLKwsI#}b}-osAtAdCKiRCTe1>j(sSkbClfyA9Hb`8_FYZw+X=8tu)K7+eawcs;pl^XE`_Z9fbR`s*l#dLX;l%O> z3VU%kd=_YfWVR;@3YS}~ZNt#@0(k@Y9`I9ZIrQ97TLyS~cnE-x{<4nKO19&i2>> zFSH@PSv6As{P^Ux9&x3k{_y3b3LF*qM{yo)^j!|b z;&AZ-a6sUneZzlu6l6Z-`Fh-Y-K3;p=QE}}wafXfj~%&^^>8l72`?s=1n37I05;%y z-tRWaw4jW;S@dOKzi0U0XMSnN0lwekKCl#1mIUT{^5a*nK6bB}u*yM!tE#-N@Hyb_ zWnzV2rG&ss!2VQyj^+G1Q~a(hnpvfdSu9W8HRzagw6VDN;oatLtNae|5U?q=JEeCC zATz*G;3?oa;H77e*eGL)-vu2vZ*}W<>2zz1Swc=4;oGXbsPGZsL16e@0L*FN=`=V! z44ltEX;vSXm8W*PO!H%Jj4ftPpH&_OehqjZum$%EA07Ws465OpWD}`VI1KCqUVZk6 zop%z;d7pMOCadMn*o6JSu{1u|1>B#q=XTmKbF+*@3u2R3fx`l))qym(@Po^FpOzSt zNfseiFOKp@K+^ zaKARR_&NfY<1&{5GnKOAZQvwu5tvNNuyerNVo=)U^P7^DcI3mmP0lJifV)!lx;14{ ze+ozs(3i4kq0FRJ#HloL9ZS{qb>I+i`q?Ab0`bEkFvTLdp1x~xpBt6`1MO^Bd$Ir5 Q*Z=?k07*qoM6N<$f-f;u?EnA( literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/info-status-20x19.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/info-status-20x19.png new file mode 100644 index 0000000000000000000000000000000000000000..d339d24d6cc28de8825cd7539664060b5cd45171 GIT binary patch literal 847 zcmV-V1F-ywP)#Fs~CPt>6swImoT4+^t93_;An!Jgf{CMxpd-q-!P9~YWCwNx(-gEBv ze9!lua|Q3Pp23k`ql5h_@OEKTCkRIf`7)MB=Mm!1=l9OfZSFt8)Wm4RJ+Zu4c6JY( z%xd4c>O21*^xZNTQx7v3WXvLwZYO4?evBtuetP!DSw9503T)V`OkeZe*SwD}1elr_rDx#O$dX;W=DTy@n?|9ki=kt;GTr1=N~L7;hC~Cyu8)sr-AZXj z2VMl2p?lyvK=Azjd4!CI{8l>is;RAKqpHp;37~vuEDCI0a|fS%J5ER65bHnOkI_Lm zp%C~>t9(O(Iis4fsx z7iPMp(W-r|E1pQ##dj+uCa-*rR&O;JbgtH)ji$BVDvdD1=)NeyJPKt}9y;)-l>Ww>p9TcwXLtGM`Uo;^A)rCul?RW% zkVg5xhX4Z?yO_tQVD)se@bX{E#k=^<99mVwlhfMymz;S#o{ti^m_JRtsTC{Pwjzb^ zl4$PW!Oia@*vZTPOp7jj<$*P}+Ou46RV$z`xQ;Ovue zvRe;g7jlPzVcyRiV7pz&UFhgL6#igqS~C0iHksZ7>00^2z&>DOE#xUM2Hda1bY-KAHSQriMusk9{}R0x7u9wd=qK#_={(u82dkh(GW;iST3nSz+$^aY0J`5wimYTUgq}sK-o}s>3%DCo@Qsx zH}l)`oy&hl5khdUwQYKu>8v4CJqM@(766679H0Y;0UbSUP3`;#bI3Y2gsSHQTY(M2 zHka9^j<8LHun?As>c{~=)m-@efnlH-I0zi=X=}1%o}pxlOhc&J3%qArMxz;zXqG;X zZRykBR8i4fxj1t2(VR0nP?ViLZB2(C2&N%a{fe-ykB#V1o)M1#BJ0qrniE&L5=mDQUuwYsk zpFgl8F+RA}GNKX?0!xn~giTr9CW@=q$gvBX#8|)Awu~b_f1ofejL#n^HDV*X%|t}v zA6i&uoO#Q)vHaEJJi77?X8Kl2Z8PVZ984%6srmlV;bD!9T zs<{Ag=a)eiiEdj4Ms%pr=MU6Qf${kR^Nr|mW5%ixHmW0sy88X(&Z?SnX-07ss?(EI zLL2djW}EukQ($b%Xf)yxO(wjx8$QI9TavcXJyN>?VVPOr2#nb9MxQ_6O~RP5(ao6} zny@YAt=OItk-M$@@dn&Qa}l;BjbrH(9${N+0aTwqP+{uh^D`}SJj3qJ)OBOzhqNo}DoE|TAn?55azf?M*Ej_vzKoyo=Bhyp7 zm8;~XfV+^IwI4{25!;N{Oh!icW^Useo}!A>YKV={b#fPd=k}*u^AuGuZ^d@0=m^{N z`1}EvDl9W&72WA2cbl1r@YnIT80|Ytq-~BkkmPu@&+M1DlndLPtACWhUq=s zL~OJdfQfLBflH}!MRm9-C|@eIRXwNo-cnK20U77$(3Rw|J~7D9l~YsU&g`Y{+~=uP zm{%fm`w)PtYR*m>?@0e2$>Uqs+S4Xy|Ap_k`r9Y>G(#n`v{1B8098>PXD5a9v=$Ej zoCMILL-+L?ynKwY8<V^#w&Rs(BJ&z3tiFJof^7pH&##3R?GK!A7d21)y>GQ0YjWkOiL z0NKg5?CfAT*ixVIRpn;a$#z9i*8{`<7l^}^_j1}~PIvO~W23E#qCN?9KV*=d;b2Ss zz@JBA(lACz(!y(P&)ac2*3i)PfC(z;M;Jp<)3WG Z{{=-=R;`|b^|b&1002ovPDHLkV1n$QvTXnW literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/info-status-70x68.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/info-status-70x68.png new file mode 100644 index 0000000000000000000000000000000000000000..41b4679310ca14f336b400898e11370df02ab79a GIT binary patch literal 5154 zcmV+-6y58IP)00006VoOIv0RI60 z0RN!9r;`8x010qNS#tmY3ljhU3ljkVnw%H_000McNliru;0yv1B0IctOtJs~6R$}` zK~#9!&750|T~~R>e{1c%&uz|SuJ-sA#`oBc?LYz{Ay?vriz;ZFmq_r0gw#Gzp@NW# zDiyUaeQX7-L{*_y2$~8iphYMbNK8XQnvmE8d`axs;-lpavQK2k+N8;#m0_^bwC_M6o?jUTtJj|Ba}7(N+}C&thD2U z)DRg5&I9itat7oqaE|xD)Os*w$3tIsY#daqiG~onNyP3DYY!yA$PTe)D^L=#AO^99 z~Y}2d(N1fiQ33VzK|1ZdIi}iHa;N6JcQVLM9d+vv9fW1 zH4)Z?OIfnQ$Ep+%k?TQ|HaJQ<345Fj+VRj%2BqCtY4>eV|EsiqO=;)1+R0o%IWwcb z_=m7l=B$}9PhQ%s-~|EL|EYge^~twK1o2v0t_(RMtev1GQZxbrYp; zvP5bn$QF^xf{hVtu(6-GmvUU?WOg{2!k{MmVWpk3BK9Rjeh7@ND`VPBvB?A7_k*7= zFy;O)mTcVIDc0O8);=U+9v2(;nXrM4eXI!)8!x!PvhLA^rHpDZOKAt~csO1G?PgKh zn^xL=RcZZwrQKJQb}o&+^o=>S=pNR>9CxiZ0m~t%)8G6Yhz_dRa`k zNK9pc8S4qiip&zk#>8O_6*Wex?+omI>}zX!X7^^Qeej8YQMK{Yl{lQgON=>UP56L~ zn!{#6OGGKHx&=h6T_n~?XioG)-?ED(DS=IOPO{eVwtzgKw09X16U6>&z~tiRT#h;J z&7x0j{B%WZ^a(NMhzV;BMnL}8x#4nXg4?D zW=l_?RG-p%ss+rd8nYZ?hoAhmGjsmMyR0>zv$6lt*spH4CUC{Zh_zk0PKzO8+W@28 zEGQ>sjx30IH^8UbqumT{rik|PXS9>~rsL(GQ`&iF^rf%Y0iJ4+VzsEWmraj@H(Af+c=>Pf-^6E%=HLd4(|1}OCJz$*+OHX>@y;#bEoL2XHi(2@24 zVhwRv!?9+Q%2l2eF#}@dTfo`t0a*gf^{O2EbNwRrUTeY!Z5-^dv7cnF)+T=61yjVv zD5dan6||ej&6dbjHdEZN4<}n9_N(}_7YXZA#4?A%={~%R#l{9uMIC3~4>XnIy*_gP z7iZjTY4+^T{?1(QIm2@kAshRpRwjE(Vz>;gtqCwe4L4VzFtC%-#)A|$ z?k88-f}1U2q9*>_C2Sm^-Avo1lKk-PL zF?$~WTQ?4CgZ|vuertk#5{DH@Q}AvzMQnt2vSfPu$@SevW%K(fZ9YtK!+x}rNp8j( zY#iX`$|&U|pooQTS(Ad3loSy*W>CD`y(Vnbl-4I(z%2HhB@_r7Qxy|7?g#F-VWZDP zjdtSHv7d_|Ho`x)-+F-j(^4~Vt*h>bpIZB%ZFZP!e3v*gNKC=DN^_qKZ|ZrF#? z?y|z{=E@0Uw2RzosJ?~tKuoADGNQD5V0!cymv%n<>S(KMc=?G(78oNUn8@F3qWTAH z93DcX&{e%70%8)T6b9I^?~fQh_@@;5x32(7lD{~aB3_|C(U|1Tb@FJz?-LVhx>zl-7HdM3cFFg?kMi(ADqHWOFt7_J zQ(QeA&z8vcY#>t@Kq-xwXh|`<9<`MtqU)Hyxs~^=`7IO)F~3imsIDp}668N*fQ5=@~?6uU-4B8#qcKX~Yy07Eo$O zWBT0oi?4kv-_Cq(927-ltJrvRQoD3lpSamFrQtg$4IjkIRXVt7jgu*o?;oKwd=RlQ z&Do2XU_NPzqt~g$A*~5T9+B-PtZ%p4zX~+;BY!pOib$_DL2uG<>ZawW(2j>!=%Z(7 zUy`SI`HnA{DGV~S_u~xh`52j=Aw&#fB3gy)y0uD0aw2xb#=&+IHgf>(#aI7zgHozr zY~0fxZ`|r?q^Y}hGPv1(a^)>#%fm?l*oA2~ORlmN5kqtKB1$QO+BIy{WSuRG*v&Q$ zHlb7&K*xS{3owXSN79~cr%99c^5pxsQ|RA*T}IpGT^EPhc;M4)Jn(69<*f^26_It) z9PzA)`mG5DY#exLc@hkxR6pR-VTevDZ-e9J$oGwq?;An8xpf1k#T)>C?lNLyg83_$ zu#R#v-DzTC9%2W?#QiAc0%#jY8xZNGtEpt_N^X{XWh=SL)}-;&<%r~lyD#)Jxa&~{ zc0GcVDPm&3-6QH2dl6F>8~2Ne62^#)D?m^8C9I&`EMCtbnVz9F>5DZXu|J13Av*2u zJDHvp!@J`ZC=Bk#L=EPzyh$8Xm*zK}FeeS$slN88lrgS|$&1+RS|4ypvzE7ZnM(U?BBlsPLzDeY#-mWIiehEa}}e0H~ajhFzyRZ2GjAF$n# z#v;~W;+3;f6U&(fT|fIoMs4T{n_S!E}KiJDZe zyh+rYAr5PlH>J(;R!(%Ct0<+>UI8al0Hu>_b_YzYg@db*?&t0ZDSBLf`fzlfIrg$-X?4_E4RS za~B0lpblydrBYzDGXZK2Fr82Qbo;@=+9Y9Z5-;E1E(>hz<5w>eRIem0&t*Gt34=5s zCZaxd78Ci{I3U_IOL^0wWt&}!r4x+yRO8kjAXZAv0<$E0jd8V`9Rs;dM8a;^f>Kx$ z;#V(|Ee})ZA6XQQV&di;QF9i=CK|F_XjiH&qQZmo*RS>_MHlb1)6E$egT%b8~0UHNPgxSPCCT^~&wrLIdt%+#P zj#8gEO>_2ABG`+go>=0bN*vCo`*c*N*>8YSV_JKcwe|u4ee~~l7{tr~Q(ZNem4n!r zus)Gwqm9WW58A3}76x1Drq@&2NgsTkur`4|cZskzjx1`Xnb=P>t2vhd-ub8{P)fh6 z9PeG_cs`4A=mw|@px#aLnoeXUiApAF5Y?w>PLC2)$J+CvBmz0NHq}fwKU)*wS1;3? zyM#3hw`alzVSSRYF_q>-9k)CKls>QB+&L%H(`2F5pz2DgvmiS}sDO0dGY}#2XQ_>! z#>p0OvL&?RB@@PM8MKo|9ldI8(=Ic_VuE>0FyC%z#=$(n{8ggnEGBB8wTE&#?wrga zGMX(8UGxfrK^qwDW*fjsYr<`a*#}fQ^C%74#e_AgW5)m?-}^rDeIsN`o5&V7B*U&a zyoC^}mbjR9Gl(_RC*Gz$@fIem-4K~>(G1AA*yz&eOJAQ`GFNgk#fIY*-cm}RA~_P% zvDPRjp?)xbmHOmaYS&KS&t1gH6v&o0lPUBk`*B;kvs!0R3@C+@E#YQMNut=8qBedq z1*V>AONTCmIuGiMSToUj^Mwa)Pk_1d>!+HBpZxY25qk<`lCCWg&!{=g)H^RQef~vk z9FQ#yQyADuu6G+sJJ>k5Nw+^8h(*Z`@LYKd+0rm#4Z(a;Q`INWVB%(a2&1E^>AX_v zHKm=`l;hQx)M5XYzIghpSwZY&P^W>5t$5yP_(nOHs6l<|9p0hR0Lgy}(R?T8Unll%;cJfC|pL>x^&miSZhsl*Up}hhs z-F>ixIg;%5cDx+qS_mHn_MdFa@b1yRW&e3!?W@|%Z$C2VHBF8IR?>;ep>c>kM z<5uo`KRkD=RjpPL`?}b;7dQa)ca__uv+SrjLmbRw!X|F6gm$uMFOQe25czYk!k|wn z2Pcz9DTQCXOzc;wT{}T@W;6}|Zl_EMNzOeT2RVP{xv#~rbob_x_%?C+Ck+vKb7;?F zMc@Ho3(2`f-7$08?qUh%$Cy0xQ=Ci@H&ehX^dktdUu_@VX+Nmah!{-Rq&|6u#`HN% z*d(Z3yY+l%p@Y`?gQ`;cSx`@-RD1tjena|V9P~ra9pl-98yVd5xN7NxSo03>67X7D zSkV=8(g!eMo#xCa&6!bRKS|?RuIN(=NLHG=tT)FWX?;VHez2gupX3Tf9zm~@rI%6{;_J@eAiP()@xzu%KLo%V1WaX8{+KA;w z!>vp@4r>b=qLtI_nT1BW*gpmN_4uhDUtL4?bfXRKdHj%A^JgGOfZKp9>tP}b#o#Ij zG+Ls(PKP*70N)3GCf2?&eeP%-;MqgTrl!Z=T)=5H?RZxy^<7Z^$r|?G->UYkMJtqA zm*YMbco+B;@Eq_;3zRi6=BIaU1K`N(7u)Y`-2dskiR$+zM_YJ+x;|Z;t%r$KMidbWSKHy$pGs(>R56xO$ zP$xMo`YYh5($4=)zWwYZzz@pFFD%EDWr11ACQkoEhHn4lG-6)^H6g@ATiz#jq3R|}W@aU`OyJO5Uc19`vO%ZtoKd zZOFZ`<;Zxt@{$&^?~;61%o$(|m`U5PHK2AqDC-sTJF?YQWN`NpPsH91>`&cmXUd|! z6p%8|o3d!JRnsowY?`@Fr0)6~;0@sX)Y<260OEg!LFaOK<=z`=pIf#62e>Gq&2lLk QSpWb407*qoM6N<$g3H0_rT_o{ literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/normal-status-20x19.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/normal-status-20x19.png new file mode 100644 index 0000000000000000000000000000000000000000..3872e2ec8292664a2de0abb7004db90e75a5ddaf GIT binary patch literal 612 zcmV-q0-ODbP);}MvK}cL6Ez! zP2{S9DYQjx+X!uHIb$hfp)yW0)3lg3&glCLFFg2o_kQP|bKbo#vRh0#qvDE^^Q!tq zief)>S`hic&zw9mg z6?HkLqR?O(lYref;k#L@fs0kFZ_`tcyaRG_`m}Fncvp{{Wu*YvU))P7pQ3F?zJ|ko zI%&X~YMo(GCJ1&E?=ga_s03;v6!ckf+cdE;=j08h3M-gN7(h2p|Hsc2FX|rVTuyld zueTg-N_&pYT>v+5$6zdSDV6L@7`4iZ(2hBRQxIX`<&`J(Vc64DmOISoc0se^bmRXDgfNhpyW%H@TCgxGz zG0cnjj8*&zZqAoj#zN5GDRd=IBZ@eJ$pBbv#!GAU{==@R$Ocz#7$1`JahxJRf&Drx zDK-TSd6Vo!dbJv?Yk_^P$4yytO>1u;>lWSyM)wTGoqKTsL~0000$N+BsqVPq0gM3KQof}TK}H0s2r&=$_uL?$UFb=axT7Jp;Zt)qZB&d7VJT24(OlN6JB z!66r|b;a#P9`_m4U|nbnot1WXJc7N>)j(tv1GpApJA$(c27H_plLb?fNs7r97d&Sm z!g0XWl*}`Dyaw`rYBucCq?j~SB$E`AIWF6wFT!Cd?I?f4 z@upt0qKGtLA6^{mE8zYb6MYz$Y)*>F;&Eh>VlrQ^&6&}q^rVT`z~6lG`c6#e;kW|Uh9)$(tMN56&o`fS_v;#h)6k}#RI8rEUqdjSq-_Q zn9S2}e&(XPu`$D@5j$~LTE_9MVT_Dn4@WIrTd^xmMl;^SoEj2(z>+aCLl#Bt?aCTL z--NeoAaiYuL(Y)qOyfGttML70Y{lLRbMw+1iy<_kkQ9^poRNqss;^j<@!-Ix`pifv?I`+f+DKR&75_wZr>iNZ$6S=$yuDSXD*CtiYXVGt6#y zJ(}&WD9Dpl$=9j7I4r?$X~dL?tlmgXoVO-?8QLog@`6n_YCa0zPap#4B8jlAoJi%X z%Iaj9k|Wd5ZMEQF=m;A!uPUKEoRG6F4fg*+#4OKOyA@Buk=!Gla+ag#HWS$zI>Lr< zI1EOUscW3==*-q0c}vqH@mA4;HK8MH&w5`_d7^%%V}%v{odXMtcnBXx9UJ4(jaTG7 z7&^u$>$F~p!05F>*Dqf1Skvi(QL9p=vLqFkbIT00006VoOIv0RI60 z0RN!9r;`8x010qNS#tmY3ljhU3ljkVnw%H_000McNliru;0yv19}0&=1U3Kw4#7!8 zK~#9!&750oT*rCGe{;^+3zzqcB#M%3QNBplg;rv#j?~DC5JkQqC}JoNe&|zy0xg09 z0gN^X(4s|)J{EA$KD0#zycg)A$!71yd8Srhue-in_8$9nn$bK+bE8*2F*@j zE3gI4L%)egO3MjJW4qDvTOk!WnA)TGH9lNao{SNF*H|z&v*b$ zwS*}@!lghrCDstI9-l1)>;Sf+*@DkTppyXbr3caxZkX4uH7XZyWceJxB@2wB_zW1s z;S)SA<1qoGahT^-3awyJYZz0Y7oTT=y=a~yU=u~!D%~eh0Ygi#ZMx1%g^^GN!_m zAe_;@qo=ALzCtHC9-+)$0uJDFm@>U&Ww1pVyt1yYTBwS;=3iIXlL8Vzj2S$BgTpU~ z@jfmS`X@TAlv)*YR%VR0xcz6eQ{Y=@zD0@s6nUI^x+q#=sx4-yFTq`CS%?gf17h@$ zWDgyz19lVen)*|9PCkt&zO_vGYaC|jkQ z^yAV`KpQTz_$2rNqip93etB*PQT;gFF*IbR|m^*}N` z5R<+Kq#35f9}w`zXrAUO{rpX1bM}^1?VqB!OgHy=iZX{NvY#A-mX>H7RAfK`ORSZ( zTuu`V3M`S88XBlsQC(rzr?1enm@EOITu7 z*$(pBWU@MfiIs&pN&4yJ2#Q`@?iyfLWK0cW;ZS~#%ddHyG6yKIpB$Sg$jyR?rehf5 zTVmZMfM|<7K}7R3+5;XQ8-cyV`G`LukIPkFWgg(h5*o9#`2Ls_AF!1Y`O{xE6mc#J^m_8 zT5Zc&nlV`h3HUZe4v?p_63IFd2B?7oa3AOa9tCy*J3vy*Fw(dwdK0%ysukcE=G@1n z#AZ4bK-?*KE47<2l_!75|^7LN=A++sTgo}fr~B?OPEDJ6+a`+$dmuLHY5SU~fmhZzV1OihnJ0t8r~#(FSGr3#KHmhsNtRxUHA!5vOeqN))&k!VK!yaQ zc+Gd)fTT5%RySX=^gO!=m?h2$j&fChPnYbp!CRm&8Ttu$h7x-y(X~KqqcUX!Faqoa zo~R@HO1x8-e6D#gEdYu*Xg1@qgLj!w|EsQ8=J`5BM$n`pfnwZo zPQ1|>tp#VNsG5?4P~n+SN60uKwCQCB1c z;9+6P(;~PX4g9`S%&uQXSZXnQbftXJPKgn6>?T_Urs|c_bmH?QnkUKACgF5*Y*BL4 zti&hJ0lxzbuHqUg0dW?%Dy(x_B*JzqrD>zUqxf9Mqh~Q==1GxbCq6p}NL8ZSX3fVF zVI3Clv%Nv4*uZQRE$Xn`wr;UWX`qNpi9s}j{39u@g>-NvZb^}2DVFvjP0g? zCxJcUeO6mpZWmB?0&AoYMI|w8EihsEHHBgepE1n8&{hSeKnHm?Qf3HEEK+`=#jETR zZRu{}8jdh!Sgx~8!1i0roN7+ihGv*~hM6cCLxG-!rQ;@UqGWaZWW^tsxTM)O z0K5piAk6V)g_||?dgIn43D``YVX`E{0Pmk!i=vM*?UmI~O(Gu`P&NSTTOgJRNf&&{ zpA_)LYKxQ&-YH;+0_#y>px`ja24H{yHxf+)VaFEf@fHqy5O_)Y#sw?g8ne>GDA7lO zehQ(Yc@!8#(Fb@7q;@CBv99-QpnK$Rd^y!Ccee)rwMc zF;Q|-M&d0K-#jM4LX?#}nyP=jHV2#o z&Ikx4>-TenI|HMHXQYA#yi`JBoIx>TzXi^|os2iax^V7aLDK5$ujS|;IH z^^rTkMav|IsBXM1)jX-$^vSiON5lpEuqy1y$;jLaYu%Xm`h+`w8JSS6ce+P--f6-D z#gW%>8O4XItPo0N5(XwB@#yFOYm}HoFFr`TDqtK5)vrk} z)DShlirmH#T`9@6J|^6uOxGBEAOIR6Xny~zfZ{eWZV?Md?yw0TpIK3c=7Di#($nI0 z@6=4R_9&}r;Cg{fdP)R)N*H2C&@;lua-^iDfWr-9+#nVb#9S(i;yQ{OHEX?kuap%~ z#swggLwQypr&e)IPna_ytg%OD?sZqb<3wPL!#GJkqdj!`o~+P3iYsWg0Nv;aMFTA> zcv&=|Q*@)7>KT}FgDHe#vF*O}z>0}J7v0LrJ(@3^q|sbsEjJl30bUqEW`R+PY{KV> zCQI1W70yV6tF6M-dU4_A)I*~#R@J}obx%60po~{#%n!?u_Nfg^e)IXFM zCmmUIXK*-2oD0Osm^Tl;`!P=6ngtq>#^E#$|0D7iE#NZ&*frpF;FJ`k%=wrU)zb&H z*>u@T2OgI?a#61FmI!cXeFV|M7r<^iVJAQ`HL{@d`&uw9NaOoTI-M0bE*T_ zFU0R^`u56N(Lmyw6XM3(MTku;#unPzxM)mjG+C+RxJ`K?JBymb;RK4~sOtH;jKX2@uaxsu}EG>St;se z6z`GXJgKU-^z0m=XXnU5#+xiy~3oW}fu>Y<>ZPOZh1 zE9}#o(PEaYEcQt>AM#5k8wsBlx91tb=a0}F0v;mTHrhf``xXGpO*W`gUTCge0)7Gf zihy&x6$U0;g2R+7~s`!H&FZx#lJ%f;+tCbG{S(^Jn9pOgZ}_{6L{Z%!ZKCU zN;~<~CIAkdy&1l7oO`^)S>}nO*(QSAPb(fWRW1JB@}OJ?P6O`%qnwl&?GWntM6b4) z(kNr#C^=vh#lN9=O{DOv+-k$c3E(t}pW|_i_G-#=e44ASd>2KT zeth2T{ISE`QUS*2cQHf?&2aI9L8{7NC|Bv2Tb$ZsQVLzXx0Z zZdQc$LHLF@c7&PcbC;$UesGu`0uBPdOL)wAGu17ywl4vQyo7)=;uJ0b$AR;lw48)^ zpXWcGZTP~`+^)+U0q;nP_$J}ujAw*BkJ0)k0qjeI+nB$~VE)dB9p=^6?n+iRRhF_=UizHq-4)<0 zaDh0N_^*YVGsm~}h5`I4n9>@?EEc|TDLkGMSH4EXc{AZz&LP6HQ=I}P)YBzyU z6cr^QQ8%U#B^9ZKSV0u@1tTl_a;clPy?b@<-uHCjHdk4W{#S?RJm>d(oaY?DUv#x& z*FsAfYYbCgZYeWgAax<6*PfEmE`;dZyZV~{a@#h_*EaofPQ(_m&4!L0(~O9IEi^K) zUK{C0DJ$xx0I=C6k>SY0o|;jeHYvMqcS(cs2QUFvHmLPn4~_UpMpGxx(bMcXvn|W; z#WIa&c(r|JrT53{dc)Kc%@Zo!o;@-vyC}1?_)Itsy!2r6G=K0zU(zrUZGr+&I30Q4 z^D+x(YGJ=A0m4SG)i8&Xh6M%$T3=SYzro?Y+(G@@{_?d=$SZBzm-)3%E0QfS&@XWB ze9Vc8rx41I=AZ}+_g(~$KJ8G6n9wFcK=;m-mNKWu4jE>M zicO8o&MTny$aBKJe3UX7?3L6UdP#QH3;=F)9iXTG2~ya8slc#?v!qfg7&D~EH158s zB4UM+Qc}IY9jCT|lNB%GW!~sINVm6x#5TobSrJWw^)}HssC}aTb}^&34wZY(k(y!$ zprNCbZg2ZvFU)33?UfNL-1hrMIHFO1rX9M$=I8~EyfF~Q53`&=lcsBJD#<2g7T zxvVNU!n5~{^aniuoKq=mSE{zYEf)Z%T9&LcE#uyJj;3W`lc^KVXNt{{9?-t&@sKq) z7c~Jc{!d%oE>8=91gL9SjN9ee1~mT%>}U)Y2Ucx)Lv3>*ZkNXeqAW>}4lH)NJZDee zD*E$-9qr{(j%|5G&CNxVqve+YMZnCc%R8VIXmPtdt>Z=c39h9nwTs4qTmS$707*qo IM6N<$g0!x8`v3p{ literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/running-status-icon-35x34.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/running-status-icon-35x34.png new file mode 100644 index 0000000000000000000000000000000000000000..a3074c57faa3da4aa63058a17219718024debcad GIT binary patch literal 1955 zcmaJ?c~lem9gj6ZE)ADf%Ob->sa0}ug+u}ZNl3azF$OmRt0b8r37AYshE0gzQv&5E zAZkR=r+^@8ta&W%x(mvBi>{VMMWlFel_`+Pp%>+{|e zCv1pc==wWX91gcoz~_jtm4v;ZvlI4xSdco6Ez3}@1l^3Jp$4fA#>GI03QPj7kBcdWS~OGP1$`LL_F>%E?OM5wRZu>hr5Zmm`h|2nW@}Z4{qZ@zSoM} zD2sE9Od^6@Gw$2`8HOK6pRN6uqW9C33p*!AdiR{0>}Wfu-!*x+x;jK&YuR5cvg>|Z z9B{7oMklEJtmz41qABAa)|-p==T~{1O2$1UK5bm}?*4AjbL*IQ!MD@tOYYmli`GQl zEO@*{Ie*-xf5_;*bntblpGmMKCvkaS1uzv~y!|I-czUSwb=moSs6Jf4fU`W!10U@^ zEGjrHR@QDAfA>Y1K;xqtv@sezk7PMrZv3K7GH;clYj4q;l-=E?oYBW-Qq8=+2fE?I|3}><;J5&+8efWHhPw(?LRR5x{5He#CVE)Ua_ie zc=^afzkC-r)OhQKTM_Z~J?>%JDJ|6V;8G1?7;-t`+WM-{Z~h?tC4A;;KX>`XOah6g z_`5wy{$Nk?^tRa3Io!ogSF(0>txbL4y`ukQ!{mXzrLFM8%YSNH9Z|R9H%>%Xj;g7o zC1GPl-Mb)NV{^gKiL;vsSJ41GG`s`vex$Rv$n$Td{=(pfak#}^Z*QQ@u zygQl-rMPCs-?BMwQ1|%Mm6cc5r{lyC|LpsNllewTaTcj?-oa}l{*=yOVA1U-{9qtp ziuH4eySu?!PN~^G?PjJVxtFEyo|p)g5*Wt`*JFO^iuop@=0NAd_wr|2!Efvb6Ex}Id_F<9HGHl;=O zAHEZoSfRBwx2k#|+R~#*ZoV04d$TXJZQE9S=Z$OMiR}m6of~S}-76bM{X?*y_T>ocQ37ILBkW85)MGKw}TzH{f*Y5o>z4f{2x;0n%RPQhQK?j!6 ziZBAZdbnl6>N54P{q5Fue*x_Rd_DZ&_SvIntxTHiBpCX=-JG(Nc)T^Sr8`ArUA5;? zkR{NwVPO2H?=HwSi5t|fUR+M-%S(O)?MZ#S6BsSCcdd{2J@0*I-;L;n$3ma4{Gor} z9Tzo14)m?AOzbh^lJjIo7kRKdg-JltPW6)E6^t|dyZm#HYQJ_iF}j=Uz?XNw2YQom z72TE&g~wW`BrEWCQ%|&P&E_ln6+_EM#?pUpIZ={*#SBLI$3{000006VoOIv0RI60 z0RN!9r;`8x010qNS#tmY3ljhU3ljkVnw%H_000McNliru;0y={4hT=f$MpaJ5!gvY zK~#9!)tqT;T-SBSf9Jk8D=F?GDasNld66vHlw(;^Y)iJAIzd)1NP;@Co1!rSBq-7% zO`630k`FDAI6=`iX+Jbwf)t6ZENCp(K;1YooJHPb+43eYvMq~}D2Wnxa%SFg@9Bs4 zhNdWr8p@Q!9$-k~&2Zj(|NA@3J@?KC3t&HZ>~;YD^rl~&_BGp|`n(qg!_BI$9asw5 z1~dR!pdM%iBH%DE3ZlR`a8yM4KXb!RkDclKRh#QNYBf0*Lj34=cV?Z&bwClwtGaqs z+oI~49IgpmqbiM{MIfV4K(zrO7UTrTB&dRvMPx|CJ8r~3YQ!4_0-z#L5;5hww%uPj zn}Ir$`7E&Q-oHa$K6u|!MXvx|1F{-)C1{u88dPKB)OxBed7$Fd8CC0psAv30FcuUM z#i(IMExp=8+yv(Ux?bNMT+!~bC zin~bFEm5^NwIshCih?>E4s{L$Jjozp00j)jco^~VjE~@zfc*$>31**&dCe3570+Z2 zfsBeVlXq>sFIWhI7KE7Cm77)F4WQR5Y;Gg@3JTSq zn*1#yCykegAP6D`BYwdcf0bvl%RH0a=$qWI@w|Rj_Y%l&)af4Jz&R&ouAug#-`$ya z8a1i94y)06=j@HDc8gQnYh5I<3$QjIb|KD1SQn$X^n^0UrRa>~I`y0}KAy?qd0Bju z!#7zx$;O`XUq|G(BHk{~WM1-3?tn3w;ZI-x!|-g$@#iU`0{r^L@5tfd-93uFSJ4lt z+ABjl*&M~?hS&z;T!iA#)Lm2+5dbsCNXQwFt2%5dbSz~=JUsDz&tz9;{kk^aFTB^N z|Ate$Q=L8lyuF~rOp)STqVW56Og(qs&OccaM&lP-jjnZSZ&Y<3ur6#4<4Gc0CWMmKi>9}(Zoe-C+Ez> z=sRd{f8w)sQCz-BaW`2PZHsLAvM`=#h;2wUu%_AM90>BxSWde~6veubN@R)R5}wI4 zWxT?se5Rqplgw41+qXaQ*=K?ByC>$+TTdco$5Ve{Cd0wzz)oJ}G`=Oa;SI4Z_eQpC zf_Q@12Ga;R7h;@|Ay7p`Fp?qT7szKC$awizMf{(6Ci9qxxA&svO~-ET{hHQ7j;ZCu z6XA(gtMSLx+0O;h?ohMeX0e^pA3AM|Wc+e#KGXDZgJ~9{gH>V9NX!hy6kz9@|K`2k_t0fd z?TwKwZ;Rs6ipW;5E=1LxQiv@^b82xe2GsXW_98*A^~k^QpTGU1iZR}scWu3I5}>c| z7qw-jpFHuo;(?>TTVpla7DVG)B3rq%D#~ot>0&iOPyy#+%HbH+iUDgkvy9?IAU8tkow40-h-I3 zCz;*%@BGYgKGQIA`{sYLGd*WkVj^4WbZY-Fwv`)WTUip>V5&Y@R7A`IuIlM)AFC1N zaMX)!&{N1Z-)vo|<#6ONVCWQLe(>1sQ^f4p^(DU&jkb-H4z7#h^5wCsG+Gy-Y8RjT z2*j}?3@j-48H48=Of?_qssEtrh;13rwu~!W8^jY6g4tUYX1eE`gup~)pfQXmx2o#a zAf9Lr;z`^h478HodswlYI8x%+;R(h^1HvFiRWWk%z2XTU?~r*b#;f7Zzx%TY)+cr=z&AOl)E_V-s>_ zxo*`YTgA3wRJF%sGV5RKfB4XwM<0E=I&JvDW4FIUjH(jaa#s+KUmd%kS8+vjF_tJw z5Lku=D)@0H*IoH(-gnL2bTwXsgslW6sFVYoQ)Vl(L=u;ZY=ziX>QTKeip$pp(Rkx2 zS?Rx?{$lp0Pu$(1YA=c6O1E{PNm{Oph?XiTfyIRdnhV|Zv~Ol@&sK`LMw}J0NAesQ zdyYzQgxCj>)5S)`xfm;6opn)fXeW*d=J&wSld@7N99*I<-k{2c$W{v01xXKO5s3my zn3B>`T*>CkKhE0jtt@TnCZDZi)0H2lt+k6MUvwNAjEO3PI2Vx3W$-+ik*%caCWXkA zFVV;yFvf2>g_s~3FRJPlPUH2?#l>V|xJYQ3K}t(;B|Yt1SkrwyJzc$co(I5^x@J~& zUB%Gw0G?+U3ZEz7D3UhHX5NcP5{Rt}JyqA4m15_kFWp=0UHOS}Tf@bg2op{efi>1e zYe4JHx~7=w-=0fSKEayq>si)z@suc4t1mROY5j*;*4f2NhYkHByD3LQSXY^)Fcnml z*oJsUiy~(Av9Z0Y9^HHIexT$h&Fr9F)m>y=)CKd5EoseQW0&;xO|=DZ?nqhHwc!*| z0K`CD-4a^0gZy9)F;y~7*L+e`EvlxVs;h!%VwI8dQJ`c-%l*FMnj%+evMxZ;+8s#1 zh~S(ORAPd1oCu`4prn{^DVs0<1lz8@V^&h8+hlNpfzj6)n%K`|Fo1Po+EJ9#Hn|kd ztGb>jF0BgV$vlAns~7&Z9cXbbs!P^F0f zA|OK0yEQ z3k1;^p2^hc)|#;{Y7sH5KnB2%Y`GJpMZr(psD?L6Nr^+nR&o^mPL{W9#HXIYv4c!R zhX}*Nly#D9&cpM~nZ=nRCB_9sT8rIu&x@4jdHeWI`iGxqWU`NpSHu{4? z*aG4M_^}JyQEieM*O(-oyj98}F3e$~Hrktd*|hFfmNs=WGI5MI4?WFOul$U`;Xzv3 z^AvoUo17_9Vup?dwAS^odEF;j+kHJ9t*hrj${s4wC>gJKX7@SyeT=GYuufW}N{n++ zqb4cZlN#R!^$=I2=y%ZG*vqBMucv!iFZIPn+FMp)jK|Rv2T)a9aGWyb(~~okl)4^z z+PAQ_`+9noZ%8JDHIcGsUZl)q#;HSy>!eDI)3~7Oa<#1(C{7)o6e#*_bTz$)t1r8i z?#>O=6&j|_v#jl6wq1P(?FUxy^xmH{H2V4!IWv7;PeV#)+v+(*SqM_5&l4RM!Fhu5 zQekXXQD19FMVjXQtY4tHa4{=8uA--F1NDXGlU`R>Xk=B_1^@;I4anfa~`RE$XSCPtkKs#2@6*2NU^^{iTPC95vFl6-ddLfP{?mbG2NjT=A3 zmUW-Ri`p3+2&j}|thHEcsgz;{2LimPoz3e$$qgGnH7`gJPEl^Dwyp&O(W4zeWQF>BkiuFycw@`RjYgMDxmY;=Oscq!?Uxh1p}+vw@IP^8Qj zKa=Xz7YIP(T4p##3}UiO27|nE=zkcU>}TDIh>Kb_QlD?1HPsstI@?yW?dm(|>|4d| zS0CcYz-s_3YwKa-6(8l&m0QkhOqt89MnQFUT?Q(Et7$<(JYrj6C>X}6rI>34g|6le zG!(mL6{nbQNM^z&!`S!;MZX<@f7cN<$RG+ZF^UB@I|~~ zknwp5gCNGo+K7>`FPZY^mjZK>lg}2(x2&u=Pn<_m954wo3X%xpnapv4(L|W}wF?q* zit{3;R#DC`DH4mwh=`1k>@~*E`i1_TC4|x>Gvdam}G-Brm5I%t&4_%!Lt`eFx8kd z*+>8I9^O8_lS6~M87m!4x31Pk%AvvCCy`=pgfafYbOleZOkl+Hc_!26nQUdM84i?0 z8w*_r8wwo(JemlT@$w~wz1D^+oVv>u z8s8O};!o8%uO0a%T<1V<9d+eY*k?69UOo7| zhp$V7`LhrFrmU*_zVqMfFH-cxg2goBk&z;mXpDVF9${eYRXQ8jQI}hq3Ui#Jqk9>h zJV+%PBjcY}QUDn@Vs?p`T_}?$twi4WJ^1k#K}7UhJJuWqc@?x9=&Uw|&pz0vF~@}r zmkv@2hY7+l^4VlHv=WXmH2F3`JVD0GW5nk?+mK+M^-XS%U?!>uzxct&@mICCP~A{u z#2Z!7$5mZ}s=FCzS&(rNkBpa30&V386T$JcofxAi883gHNtp`$M-|<3aodeA-}>Hv zh)*JBdCOJ6qp2|d80c=RQIo>@1rsJEM^VCfa%w>?iACOpASDF$gX{s_Ulrxg|2=u~ zyLZo68TiV_UMQ*R3->;_wjS69EGPNgqFE?`5D%#qeCKyJ`Sg{-hGA>!do0s`gD_C$I|`S=b^|(k=Gpd~Y+Uo-6`h5zNkv$v>Cz z3RAJAtF@P|)?TJpzD)1n`CegI)ji-eo=|li3*}Oaz)I<0|4MPs|K&%YJ$fb;XSHuX zxUSc_`17DQ11ryVs<4RcW#H$)uT|Z%e|`I#lgWLq2jHu>y!=i}_6+HyPsH3OB0r&q z{r8L59MA_m13UseSrw%wV*c)-s{r`&NB2w}`}>DC6@p;w+T^1xY$N&Lxc~o5lmo!y zz^{NE_uRH`7~q=^Tq=OxbKAaC{^^X_=1U*kRR-BBB0mJ#PP&0_5!=ij$06WxLGJfV z=Am4sVRVkya?)I>YE@G1eeiMszIMwi$&38lF>9kIA-1YC1J?pwK;0q~rs@TgBp()i z26)UE|EaIuviAhQ*Y7U?FdHc~69e#-k36r&^M^ooiI{%|`9A3#~J{>e(P&9k8e1&C zzUQ`mQI(XOUr$r9I|&mXd4`%?X&yz|KE(Wssy+$&e&9NwcM%A)ANXZjobCY*Rz*2m zf1K0q-LW=_Ik&z#^LUd}`v<`Fz-FL_1JUhd~_ zzC4^H!god`2V9b7tPSbk9Vs~lAW!loIvK!AMf1|f%*AE6=2Vad zplOy);TUi*6~qC@Q+It4cox`y&u#lo5#k#^TZ(V;$I z1{)(C#5H0;ZJ~*cmJ13243cHtUDje|advnZPBNJ{@7{NE&zXB)WE(u-bd}3x&eiCS zD8>9}XC`8Hl;t6e<<3QoMI(5h&JwSFadipxOi~@Ya0qQY-pi9`RJIUi$Oh+_Sg|v7 zQ|jPom8C|dv6O6tk?%|_2M*ToJ1Zx}9D9lsyJ&5y@E)B-7Rmv#jihoBM$wXs*Kv@a zC+JzF7DI`t5ZMvD#{jON8pw&TMAssB#t;v2c8=bBVin_Qg>V3;{^RE&FCi{+DGDUK zZWx@A{sQaU5U$}ay>Z0Fe6St+Gv9G4tre44*+%*T$KzVkR!wNjm|w(glrz#t)1Q$? zg?({7>4+wj(*rRcC8m3t?rq?F+Ta1-*Wx-~Lxg2~PI^s*j~LGQ8V3kln2#%b$OqOX zo0VKMf59x4@h3bvpWA8{e`5X5^t}gz%9HD$&d1PT{DIoWSfN3xR#I(U~q@G zR|#hdJns)(u&I}(68Y`^kzYVhn({48nJgslE%*z70*9umgi2)q0000Dg-^SS4D&bi-vjfhaM?_SzWGYw?(tATaE8lVkW4GaS%U^pXkkwqx39XOE9cLIBX z=PR(G3{-%s{akLxB>>l$0+0jF0H0<=s+#{WVUZcg=F`A?Gtg6n*aT1xS=0eqfEB<4 zFl?YVBXaJ(U~OfiM*~P=HnqND}xEc-{6DfX5;x`V24*J+3k9=fSwfbQYkernsWz7G!PR zxwW=pqNoV5GQ2wvX11Ye!g~aGxIv?P!CtXe0u-RzH6}d=a~F1tVtd{Uh=^aW0FT?T zTHS+G6}AC5t}z`G&>1%AAz*jNm=@qLu+o-sgx27}%)k?xi93RtQh?IHaj)B|6M7Q5 z0GzUC1w!|BQ-sjB0f!>MOj!n}>|IP##ej~0-|qmuz_EaH9X7`_Kov;2#w47IqKXp1 zWy#yX1>na9^SXULCoFFSQVuYo2;%||lL4;V<14_~fO$#aN#$03Ao7pnz)j8ItG?qg zAYZ?jP274tqGP!M;4Fj7n$eeh$3Ha4g;T)MdXJ)5J7yW2xVP=4unAxK0LGh)`waNm z_x%qAOai_Ia6S?#Ii4kKsy~6Re6FpT&iktE7X?fbKA*cCB;lkg$$(onDW^mF686!c z9KP)3(;^q;ZlB8*!Uof#C_75b?1g~&w*cG_tFFCeY(YNN;E1v;?psUgQJ zijgG-=^Yf=^|44v4HIiwLEm#5N8(;f$t(fH2(W!nq<`^mA!;?IZjI7^{z@0{j&%i5 za%mVi5{DBxk00006VoOIv0RI60 z0RN!9r;`8x010qNS#tmY3ljhU3ljkVnw%H_000McNliru;0yv1CL_yo5#Rs-4!B7~ zK~#9!&750oT*Vp3e{=SDZLjZ_B+d=ua7oC;gi8uZ2_Yg47on;sec+)_l`6GDl~AEY zNUd76>SHBP`_NVuYE=oS1PY=N3Ly;yN=gYN0TKrY!3l9f?v5RMy=QmNnLf-n-pS&% z*LzMVBaOWF?w&LApa1v&E;Gl91lsP+8vt1Q!fO>j^Ejlk&;iWWFiS%>&lg%hBggt@_rrA zs0~O(1?PYkM8o0`762^~X2u~9Nk^z*U#FZwr2tP>&ojSgfdOT}IPAg4J{`4fcU{E5A37{i{JApMCZi!%F9vW>Aq27X_rI$BdVWpIG&-BJR=&ggfX}Bc~ zLmEzMH~{R5U_Wr|iZf;^QF{a$LZcURim<#0>m#@=guWuEK$J9J#*#`mRb>FVfLU-- zvXW^LrnLFJ=))%p-tb_L4+qk4ECoZqGq}?=QTj?705`>JwO`;dTx8g|IRQ z9XV*tgDRAnhyCf*utM(NBcLcC=St?~r(tmhy6fR~;L`}+((od1G*QM(FeNfmTDxV} z#CwlI%Q#$9gf$VY4`EXgIy= z9#%@th&j_UMwiI_GiVCnE)91TU|k-rA2TeO5L4|iLwyC_Ld!yo0^MGz?}i~1>lVfOH^6E(#^d`=&-a1o`$XzbVbmZ zf}s%VABO&AaQb1;mMK#)$E#)#L(mn%<`906gZoF}h7s^aWl-K%2uLN6Y>&jGFPiYV zw*cRd;71y6Jpx_7sdCQ#RI2?ns6)_p32rIE#yqSWhaN{uvdxGpH&Bo-)^1z9Kf z>jtiI((57MqqFD6s0 z_h=JyX#wT~%jES4@HvL>!vaiPVYWG%g64X-2^g<~_a23z7P$B;Nm^aYnUpbE=!xL& zJgm<_OQ9@2sWmVQFa?%&;96iMCevIXpo{~@MQ_rMNhJY}WzHo?6<|>dY%GEvf!Bcx zUt-Me%^M~d^IPyoq5A|Z3*d%4v=vKe$6`uBBGXRbYT#yIl`tjk3_K7vr5v3}B_Qz1 z83)ka0Bgoz)PsHtOhvDB9@;{<1Gpm#9r>~(u2!ZLgblNSZwVmt1f+D?XB&+dnV3k@ zP2#S`9IT9BC=J`Uz>)80U2xL|-$iNEM2oi-V08gni!nocmnrjsK41-SLj~1W;+( zu-qh|tN><7A&OFBm?$t(e$6OY{2BEA#W+R-OaRR}SWq-u9Ca$D{kBT*)<>D5RoOgY zO0QgJsem0O%p4~tYt+y?2E8K|m=oY@Xb+%0AM++kR<~1D{CbH?YHi)X{lI;~oKRM{ zR#UGuZf!ON7v-QgYr-h}Kf`PVokeJhY7||Id|E)656n$KEbAp*2qk|~n0?xD5^wbp z%nM+yl1RZDG4tq-%#G>wEEi>0R%9CZ)yp!5ZouXG}2rJ=3>odI+OB58gAJ<2S! zU#p9@`$i4GBIylDE~f*?pqSDZ#+#B(rleXIL01tvmG}xTZ%!Yy6JuA>no>QmPv>2-zpVbj!*ut>XKw68z;br;N@z)y0^m>$!|k zItD1VvggfUVCR4x0?G*|Hd#<*d}4E{Vc_2h1%2{;92SlE zNc1J;#3*MZvz~6H((<`ZClAvxX)p88)&c?@BMnhb@@T96mD&if7qjDaT9{(LKZ4mA z=*OH2imLEZ^%CRsh#7|oFej9QN4{sgUM_BwZFc~sib^j5hole7Bz&hnat?C>$eQFl zN*iyJHD78ropS92BH|#Fvaqj8*9_x|Z;09X8w8`u>@ zpskG$3459=EQp#U%b($*7L_6NMXV_`n8STn>9X@`Gi8gCNuQ9+y4QK%ZmthG@rGC7 zI#xvOj-&;C-Xy?C1#UthV|b*zf-}mjX#v3dIUFs(DLu(S;+X*BNvM8QdQisG+zfIX zPjsaq*ZK&vLpfY#?~wp#bs)z6SD@fz9h|6>p?eD<49#Mmj{{6fCVf`i?zyswmLFwR zRa`HUNuLqHKEvg-&WUDSlzYd>X#x*EuQNS|bFHj`W6H2Mh~t&A0?L2@WOFFr3FOoa zuIUSN28A{5;OxE5%V%O7I0!r#Xn@a}#OwRALSqU()uzVOrejnMw5;GE(S#P!jW$YW zV2V|y5T3)fOVT4JCi+5jD=YViahiKr!_nDryxRtN5k zws199T(~{;P_2tqWv|o1%#THTMu~m?FVt|R0E15&8Jn03&5+H&$6hHYtyya<^-qb* zJs>8K5zo>kZrrOP)D9g{ZIOHCB=*T~<`33Jj@xNc2p7jl7oKNJ({ms*V7;H$AnbxFY<5B8+w*x3YQ z{#v4rj^KnUdB;LrV6vFATps`j1(5ktb7^-1JzXIbwS}))k|;KcsJ|~U&TxFuz2L## zH0)0ss{~-fn;+qI&bmw^hCSHr!GA>GVg-CQ06U5~*uP5(QucgIhSKSS@@%^3qyyJW z9XT)8*eL>EAG02-P5ichH5`1YoH1>yZgR8LkrccKye*84WlmNnhQv#KC~kLLR=>?L zB~_tRYQY;6^En_u=ba?9mPi-)Siyb`1HX{C^Ev61d$_HT)mndWk5bLRI$?fmt@l^X zibfLGY!^4)BtmR!F|N>-$3=&wMw68~PO3Ifl%N6eF&6qJu zRu+4QhWB4Gm`@u&eW~3GeIfi%!$!;lpYUI0lP7Wo%?r4jJ6TfV@o@<%am#v!xrOR z`xX368E=JJ$ibFY^MMCPsx+laZ7b*p;6em@6r9)ap@#2CDVCTe*DKNT zz&m0(JAgxJhbg|TkN()tCbLO5TioA5(VfPx0VsKyZodGs z3)o{dWlO68C9^;((kBPvF=mo>*n>Z5cw3tzD{CST-6G{ zvfN~v_Z&QX+21#r|L(B?_$KC@^CC)nV2xh^5IKnfFNjmvZ=5Onb~sMLdY^kg8mjuj zvD|J7Mj~U!ha^5&0o*3+xsK!?1#nxziB0wa`xOl6(w-jGc%MYV1REiapFG8*=j6G$ z(yG*BPG_tUQNC8HnT?d5wz2<~!TzHUH_j_B-W8m5swj1>ob(aVyH7FIfcI&$V{jC zB$L9IB!NH6q_B|?DuCl$Dt+1{2(?zf8u3|M>rgW=e!!1J4^icGM`M=6``-k;p8B9? z6HJyt2*6mxj%u_>$uUBuPw!P5GM`rU4TThdF|hWAHo>vlAM|=~7&%f{ET|1x1uPu$ zMkA#FP%|)&na-w1yGx7x!7aCff+Wju`-Ocvd&1jSQU%BIy4F~zIS}F0_1Y}KzU-C+ z9v~zRjKQJS;au+%sF;lP$K4=m%|TSgpv^IE>Q9}?3;o%WC;&+Ou0;w631EQdL>cbQ zufcS?3&$I-AtcF4-h@DOyRt>++TEIgaZqb(T{-VqH6_~p=2Fn{?sM#}H)Md7RVZjj zqYStDC9LoGlT&xD7f~jKk0t6OqnZDa>Wc;6`774 z(|I^wJVUZsrz%bNaBkeT87(mk0%%b-tsKc3A(3;E+yS^@w)uu*%fV8X4YSR66Tn*~ z7rH-^Nx{g&C|C@zX<-wV)0r^ad@UfP4Tlo}K&jd0eLhxItX!}F@S!{xjq^^7mKbz< s^xFVb0La@j`31lNVAyQ)S(5$o4?9;coJ$+U`~Uy|07*qoM6N<$f+-<)p#T5? literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/status-35x34.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/status-35x34.png new file mode 100644 index 0000000000000000000000000000000000000000..e1594511a0fb17c8ccc44e4faf699e4ebedf57f3 GIT binary patch literal 2218 zcmaJ@X;f2p9t}hm1u6kqi}*}XC`$4|*b+7&2@(+uVJQe|wg(B2#pDG-Xb})p97I%> zU>yVj9hXLtC4=VoYyQ?Ff|Ge6UD(PED410gtqk^}RI3fX-wTQvW1!4^wf$;K`i&>mR2nN_tJYVRIetoqG4e+_%=Kxw z6lOqw=KDW|rQxY!2*-e=BAJAv8C)D@F_f4@mOw06BncOZl9swi=ZRpElqV7cN|Am-2M3dALMhA=av&PT8?E7E`Ft*khzBVYB7qt}aVOyMRJcsaDtFNc04pSZgH&tHRh!&jdvEeUrehxe6c+w)OcN@F}0Dd<0Jy11JEdb;qv~vsI;vy z5ta)I(aE|UL2Hp##jQXRX3Tt)vn3on=ESjvd%p9}+s$(`y>r>OE7lIxY?|_X zIuX8JU%RWsz;k?)e`G~ud2xmH$c)>=OY#A3&CaZ=#&hVTAo@?CC(ZkKu4_uQ%2=C* zh(nd3D=UMe1?@ri9R z1>G`FzJo)XhZZX5Fa>ZK&+hikC;e;CCzj`mC-5%ZTDgZnnCx%a8q&;IQBBWk ziM2WXDB7s4AmX@{->jF*sGaVOk?paDE6W^kcNm441w9qRr;)p!o$7SX^4sfvQ&kh^ zHdIn(DJvXi1_}^Ulj?G2YbRw$r)+mP1U`6jcmp-g@ibm%^;x@hX5!Z=yNsu^_IPb3qd^6^Y_Q2YgYZOWSNF}+h` z27dt@H{PvzoAz_yjVpHGY`L8?e$bJjqoP`;VKa@3h~qOwfegj#gucG{?E&9bM;hE)Q`;k|ZB(%T*pmQ6p`TITw^w-x{q`9t;Yz-Fzu-M>@wM_QM#x!}I>mX?$_U>RaWg zPUCf#HaZ=m9vLV5;N~A+UcLVAOvb{Exw>a=FB3Aw#rbU_FvDe)c}IC6-r~6KHxnJP zR`1^+mMt{2rhm9-)ZCW4EzF{#che-Mo)S}k_WUoKE5feO^8T3|Gcr^6L}#+Lm@8hb z$D9?Xb$AH;*0xW(vwg_zX2~WKXMNMJ5q_Q7c>#C!uT@jEhp+cu^LTyy=zRjy! zqdx{n`t8%LUT(_>wG3uI_i=Pxu}u%Wq+UL)grp7LB@Mc%sH(=(=0C8rS1)8&v3yU2 z*RPpq>|*YDueIDWv*LXB!iag)({BCokzWg|A`h1KD+O5_l!iUTIj`M+lSJBXds1*R zCeegEy8A($NHG+uZ}^0rw=UsW<0`beh0{noYF3>vG1??dKBF_$6ENkNOzN}nRbkb2 zk5-1%$9tWd4*z!q$Kd_>sMp1)13H;63%$;H$z<~;`xJYX&x;KXuybm1W+|l*wq_)C zZteZ+#g=V;M~+;%Ho;00006VoOIv0RI60 z0RN!9r;`8x010qNS#tmY3ljhU3ljkVnw%H_000McNliru;0yv16cnDn;GO^g71~Ke zK~#9!&74_~T-SZxf9KqLdz*d1%m9OB0FVF(fB->|1V9p8WSXK}N+K1dn28mYEk~~S zC8<>M=Ahoq89B^Bo>Rjx#?k|jEkBRZBG5|SlSGHDP1u@EEx5&#GL?94n+JOwv3iJRma1$s4f?@_3RZ8DF`1)I8OWl9A-S0V;+tlX)#QWd)az;$j0#X2# z6Kh&ArdzD-6l*#W=|H4IgaU{M<>j~|8qsnp1AR$74QgxK} z4}R}2gJuS5sphiEZRE#4R#(6KU)G4VyMgT>TM*eO$U4NfV~iG)IAV+=);fqN5v#=* z7nEq%jnR$)0j;fa-9&p{fJhaw<01?Lml3(Blw1Nn<8x?gHJEbzcfaPCpxz;Iyhcp2 zR*cyq*6c!LJ0e@e+VzNS6JwZrSp(LB2qG3S22g0%1+CK`)*9uwDAz^19!l#vFpN^4 zq0~jColmsuo>yMx22e&jwS#~1<8U9^ z6k?MY<=0V;Mmeslv$-v9OKY#rHxBuB<9!=9pq_itrELnK!$?{dd}u z>daP2;zMH0^J2}*Hj27Tt%~vMn7~Jp4Ov99Y3gu{YHiz5NhLg*qGudh7 zddHOO{6Kq|qsnn^9QfXw<%NT?9Ok&odyrX?eYM5-^%oF%!N$>H>-*c1>P)+hf;mw| z1T`z3s|m;=2C3OG6_6+*HYH5GiuN*{PQLJ@+uqTS=RJnCub=pxubl+^l{4n6&)>$B z)BogmbTmELmDH<`Ns|0+8->rAz~5v2x{j+AY#7cnq)9LqUL{SiNW`L)MtL4?zJS}_ ziIdA;P_FwwmFvB&9Oqp7-lxZQ{lUMMo=>!HMg zi6QdABr6hdF$SB&AmZvwW<797XLBDOeeH{Za^1@ZzWd`Ez_q(*%w4V0iQoIXg{i;( z*>)R6hm&gMfC>DEZQx_0s4*zbgQArpHimK-G80>BC^CKB^zV9%mfl{7Le%6K`m;}v z@-#RODBZLLQ)q0G5Le3BI9{9Sc3-n5?ond@8*mBWwRiQE2ZZDoqnzI!Acx8fMz0rhg?? zUn#H_69k}CtMUDPDCH|J^TDy-{z}nn?=0>6!|#~IGv}_1G4N)RWE)_kP5pT6Du>3{r5);;zFotw6hZEq*3*O|U@nNqb(lF5)*3$ij? z=9wc%91~ZHj*WsLue0YR8-)^;rr!pp?_kV(uf90Pm^1(U+iqMg_13Qq?=*h>5gP^_ zHjWUJG|XW$k0>No2&^S2w31yrM9agEuc$tQyZHdE>psR<9>!l18 zVO5$+Lkbi4AoM!<;E<`#RB@bhv%uWbDuv@C9g-x^f;?+#GhH?e8baGVC}M@cSjJ*Y zcGCuS{H?Ds_~aov)^DPvw|5SdlvFl#&I+xdHO5*Lf?6$P0E|VFByarsqhf7QXLCn} zzxzj*hkomKq6NW1Q?i6ei0W9scUugr!G z@4fosJYz(JU%v9hIvYig+tA;G72Ry|QvphZ$OsdWp|EuWyI=bz8xB2B$Kd+Af|BN? zG=;Wya=m>-g%*6rBiqmi8h=+#9Gy-nzYf;6I<0L_o4}v3QCM6UPq_dm|M537C%*ml zelg}D+@*4-~eexML9D0tv?T;+$H?o~wc=)=*w_(}8TK-P+EY1J5$>?2EK-*o5moSZX=f+RFMzA7kM0r!a$?n8~*hDuppg z>QAjwUPr{H8Nnzlp!JrS3uiZ9{15*sKc_J^j9SH-^%6(x(i=CYEdoSnvR%FOJ^mE^ z`wx)s>1{$EHQClyx*yt3-xE)#!QiFy)QXejmBrI~b@WCAF$tn|9da>4`=dZz zSFc}l#oA6A1f3>|Ky1@DvDzWZ6>$33F|g-x);{tmnOwf<=Wgp8V9!^+&W^ACJ+YB47*TI9sqAycn!PHNR0fP3-Z{Xrj!Zkxm{Vj7wJ2OSL&um+VY;CA<5b?e#xoBx14fB#$Ptve{YIbzb_a8*JE z!LxDPZ9~7uhQ0^j8s8s4(GB9#3|pX4NleHT3bZ}6osNgL;pB3w28^4{(!FUjN@*sq zUST|rNrpcms7&K|9*$b6^CzB|q{l|88>L(TZKI$Mkz`I&s4{vA6pwHZcE{0&iY3W@57 ziIHXlU8Pb1PGzKJmNBj*aUN^4E7LNCD24X2e1g%NlujLE^1YwqkKDu?7z7czc0ab{_4%$I2KGIPuFWuc`2xYn4Km_3 zKb=Iy6i$KKcu4S1zY%(gJ9$a`L%=hjqDZRc^TT)4w6~Zk81BsD+^6#1a5OXBpn$o z5DJns=fKP+n+1uZMrp;|^VGOfV(PuOnfS#|@o!#B(+N5q=eS&?e)Te?Q}0u`a27Gi zl2%IR@)Xt&(XxIBC!0fzU2#TB;y6qhqm&N>fL)PQvVx6c;(G0#=S8JyN@q?`K69M7 zIMpy^>GZR4L@+)|@q@o598$X_p~2R=k#0tm8CI`UR~zi-W5irwD3hCBT46fr%r6 zi7|qSF=L#nGA5vED9R|60;Ao`gi6=dnx^fYQlK?93<*bW5RBZU z(Ax(MV3HtU=K2+8Zd}DgQJV8zOm`8Kwl z2(x)TSzOCn{_`g&T~bPwNY@(UI{Ex95xY@DqNdecDU^1wfscRnGXDBa*ljz2v?Df- z38p9Urzg{8V)`EWTcuF0hnR%8I7Q7QX?L|le&ZI}76F5`5cr5+2V+)Zks_t_xYBw8 zR1BaW|4;w@ijAXDgrIqiQ7Fg3)N9l~JxBfGIjrw5s23R$zfR)U=hoxhPs)|n=v*Ef zMN~gMPwC7FD#I7?OGWGgYb7SZl%}wyDM;dU;ZT#JtF*qQJ@2OW+%UCxLtBl+NfBgf zMagAHW22C8^d|nbVS=%d+e#M33U)!sRV~Fp#{p}JOGW%!H<-C{ncBz=OgO)c*9JcE z#2E49I5v*b^}$Wmz*MNTzNS6zn$Bbb=Hnx&E2XX~rLF;CQ{t;MVhl;GN;o#c%%|t5 zUmH%p^*r!0i0d`#$jfY6<76_}B&Kq8n97yQX>oqmFRE1uM{g32k04RhSOCyO+5`#F zdf3UgeCD>c`E$T1t!v70E}``$lnPd2@5CCy=}F3Gj#D~wf}~tTdl|A_J!Cq%Fxnwn zxaN1sDVjRxLAgXwE@2zjMU@i%^(%x^#eAs~vJ3 zU1U1CKqg`*@Hhnyd#}TFZ}GR|H9iJU&NU6 zh)lB5rW80X;rIxnKl%aVZ~bpn;M2NcGc7y!klnHklgSaqF=FqY?N335;f_W|Y>)wd)>NnapjmjQ-C2d=@e0CQ4lZ zb$!;+Y2JKmlIjeV^B+I4Y)RY4O6&iz<_j+$f9B8Lxy>589gppU zpPWkR<9Zbl`9R{R6W9fGuV^}nAl48~O%Rt$m|B%wS2s>BPqxrP)YXkGS_tZQX4FK` zj!U++4NpOMSMJ)K5mc;A2g;{@RK#+R=xt_-|%>PB5e zK0LCkw-w|tFi5&@Q8&$;MhXFIsb3rB#{c>rZbv6hTLdAGh1dtnmn zOquFOr>UL)kfb_;f9raKB@aA(7N!F$N;^lC)^DQq-1@tGU*p3#xX=IO6h|KI2T(7c z9TTM#Z*K3rj8gBCZsOd(V&*hh5YLneC&uvFJ80dqE!}g3q*yGBi_>+l>B^@{iKJX4 zDota<0HvK~HD&gK7}R;RdRONQr$tN$!o5&sovNmCa#pyY=SlOdeXe6)Pdt5KgHTJFCD)!x)h3c zZATvM-(!>bA0hG*uo1|vN@=m7J+pDty*ryOZk)Ar>C>*LwDvLZ1K_X3nv-Apv{C~& zvS$E*mp`1xNiQ+V*Ob=ZL#aPy8SC#KRC|KH3Ia1b3u>gBMNa^4YuA1M<+J0H06%O` zPq-UX?h4Gk+KVSg<*)V)OeaPkS4yTt{!b4G+=3HOfDmZ0wtxAC)Zd zP{U&FZG7%;FeeY>NdFTZk95U`ZoJ=Oy3+;Ixy=fBfsG zZzbIROiNmy&xJ8}4dr2~A$l#qnua(J0b3d-z6)q;fK_M&s_q>rM;LER69GyMc6^VlVKdkKku_S#~SP4OecmOB1l+4R6_+X72>Id;MF<}$vVXhA#a2EsE7;2 zLobAv@Kkvbu|pw*MTf9UAe|c8zxP$P#=CcWecvu^==Mf`zNd%h`#hiDpXZOlZ?v$m zaLTsrKGQVMIgS$n3?al9zu*5-QIuCBBO`n7(^rY!TR*Z-sSGOS{QP{_G|k6_Lg7k2 zpRWt#@puS@Liqju`+-1UGLcA>>wt$~L%OanyMf%))YN1IgTeN`{8PP>sdA{q+dDfu znNq35zbF_CnjIY-q2b}-QWZEmI~&$C?OQEOQ4}1<`QuzD6#ClQ+S(TTzI-46Ow(Me zg%t_~ip3&|QZ0gFvB<%}L0y))T<*mF{{D^Vy>&z;lR2yF`uW2PEX!hIVuDmEg|6$> zxq*QJq?EX|C7;hf1Q430_12m#EiE)RHxmd1n30NSSpo{3){ATxS57w(A#^o8?jgn+qQpug>2g{2;f7lVObW5M1r249sstt zw<(v)_TtTd{5CzHu1K%D>6H7Vr^fB<-XyB#Uzb>P{5z;0DwRbcEv zmd#rcq?B=B=r2JAa7IeG@b>QM`VSW1&7G56%6uZ3Odk7H{t$2hX#2I~Bd`XnNGaFc aqWlDFB_VXg5%+um0000URLt?Czy;uK zDdou^2nd1zDJ4=$G)+TM6f{jk*Y(H1Ja7m2!LqDyS1=WGJPmvjhT+88+FGK~XwYmn zV-KpTDu!WTnkM`9?ORk7<%VThcXx!TnByNvDL;1{=YZ?FbXwahx~_BZ;6V(-r~v0J z%c>285ke$@&pgk&w6e1DOo;6<41-iERaI5>-EP-Z8%79`jH2i}$8p|r9EaVErfH

ho3hTjVz4t2x`Ar9B;^;>IeYwUHTlq@eVACpqv>WGn2ezdx}dLZr~zVB13)sRxg z{!pz}!}oo99m{c?2_eL>Ef^uhQO9v6;;P2?eSF_1m&;+>c1Mtu65F=P<#HSA0%K!iOifL-fm)EMsVT0)Ox)+)oPV; zxlBK8qS0ua08oA3&-PP}El{;uZ979dfvh*fEPxsWLAu}m&<(T`$a;a8LWra)rA!Rc zuw2(=Zf>sQ7}0DtnVXx#b=`rxauh|T3e*SboVIN6%6{R5yT5{U;pfmEy2wrglmP|Kidns#q}kdB?! zxt9h1=4lBpK0(n0QJh1E1se#mAJ*JREpu@;jUHFt;&Xnhe@SUJu%5-@~4g4 zfngY5#VuZ1TB1}cVcT}s^4$)yZJSc5#M07IFUx~`K7Vfmvw#2oyFj%c(89t(*FMq$ zEi5eb3qjZQPuphTd_F%B1i_a)-#kpEQpc}dyVh(Q$gf_#`U~*-^O9(qc6xSpcD~~b zY#7G*R?R<$6gERTZdI;dzkV_d!(Rb&4rWbXD>sV8;%wJFddK!`yui5wzdzJ8?G@mQ-KZMibR48B=KuRd zl5;QKtDoh^4}q6}uY27#I%E;ZX_|JtSS-%(%JUiNH}F^g<+l2c8AZ_>z!~5KaCrSe p5kLt1u|AT_1K$>l#lHuU{|CQyvUL!rQ^EiM002ovPDHLkV1lQyMwS2o literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/stopped-status-icon-70x68.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/stopped-status-icon-70x68.png new file mode 100644 index 0000000000000000000000000000000000000000..0dee5f01bcbacd5d53c7622ec15a40accef2cee8 GIT binary patch literal 4790 zcmWky2Q*u47>-iZR-<m2n&>mlvN1F9u!4(E z%t4@wcMbJ)EuMcL%XW`LETTyfTOk7t7>z z27Kv+;~}Wa?A=0fUWt-2ECEb62fe{(Le*n=0p;~UF`XCeo_Kjdt@9<5f=d5$*I%L` zzn&2eSKn$bySQ#ll5pt4@{qt^4Q|IRf8A&;TMaR&D$eEz9nhzsCZ&08CQh+ByKyhx zq^zhcJ(u%lOofp`d4k(f;j_x(t)F2}%)qMK6UapNuYqS1u3r+4Gl^oiA+(KJFgK2| zEHVx=`pRj-CC;5Ii80S!Z>?ZUl)Lwy_<1OfJC7LyD`Q*0!a4|EoKNry^w-VkpitcI z*(psnR4%qVC2^QuSr2@qz%9C$;Q4jQI zYbeB{m4ZYfxh)Un6t|3ym)O0=Jt;3i)h`@7JoV&am{eq|brL8>Dn>4Z?Jgfp?KLu_ z3uGS^)en3pG}(JJ^$|8V30}+f6ncqWk6DaTF_T8nl%NXSjJA^)k$s_~N>|S|bf(2b z??4E7Q*R+bku7D5#-Mwfi{X_MMoF^axK~{}&EzZ|#zOL~l4GS3=B@JTf!DP(DDEkh z5CNdn#0Q=w-51;8CugHomLh67WuD&!*`pWv0_WOa2TFW%k3)k#1HkUh9Il7XSSj6d zR^+Geap^W@1T+5FYnAdoB-FP1W}9-a@+CREb~<969YLXt+p#@c-VwWIyHoyy zT2|}CU)_pqLHFg3JH(lVv<5p4!ytnl0=3dy{LT`}uk}|0gs22x5vh{;a4Yb>3duJC zIOkt8Eq`|x)-yQ$H_&#vyF0Hh*_R$byx(IsSUY?Yw`eU^IE%Fx;s)m)hqjGVfBJXI zv2<{eYpUeR|4D&v_PZ#{x2ka;ud*sM`iMv?aP=018;!W`IM zes;0kWmPLn%P>oisZy*^d{l-?w;gMB$6HRgmX;P(e<3S(c?vF*LZLN#GwHT2_v$fF z3AnN{i4Xa zXj}Rme$ z9DLfCiQ!_PgW`n71GmogrJ|Y+|LhGe*G_Iss5kG#`w*8WRDNsA>MxLWxi;8ep`!a42odYaZ2kolN({WTn5$F5VnO%=YFP{79FR4BZVfH zG-wguacFq?IuRZ^^D!~@Q&%V8IG#g)5w~G?gkGtIjNzf^%eq&d^?-V(D%bfV&6O0B zwcPpyEA1L!Tj=Ra`U|ls&>}!7*T%rP%Sq(Lp5?uxW}Vl%tWMcmw1qviUxQsJ(#WqI zFfzcX{q?jU2Hl~P1FH9}5u3*k?&0jNtH&C=-J;!!R|7z>>fj)V1gI5J3;i8kI36Q z9~56}vZ4YQ*mSu@K|!?Uh)?Up{p7>ye0%A!si`KSh0LZO;;>DeIa7+&Xs=$J*#~in zT=Uq|(`CLp-jhm*Us`hQCXted6NQ^Io*L+3ralvinStw+>M+`~N0oNb>qNti7+rfd z|DWf^2%BtXEFIem>mOURS++N_+@xw?AIwVml5R!adMV!1O&9bb&q9VTrc0vy5#D() zSIcv<<~6VYWbjG~)OT$ZJdIu!^=l9@F)?vb&M0rSg3b}2)(lo6zDy7UW$}1?Aq;@^tWAqQ@)lCErMN>aA5A(CMdF%^Yg&# zC>B;$BSS;2#U@!3H7HguFE0=fNj5Fv=Dv;~Vu#Sd6&T^$NH$>U3Wt_lc{riUeMGXX ztn3|@9N`+bx*gV{ z>*7qi^V3Ks#rbQd>W{F)sULykh#vtXCK?IJ8$7Xwz#2f7VPs_NS@Z+ca5aTw3|KI~ zpx|ZBt)Ood$~^>P3cFVX1KZl#K7RVNI}jn0(wEsjAG_fa__sQH+CoLO(0fNG;D zsA7PBDcR##x)L*KpPirj-&SwP%gY-=(xWwf`=e=92L}h`&V6j{f_LxUwQGEybmv2r zg&9{;9j1cgg<}d#hTRd!A*!u0kDvJ%N&z|-(FxBU9)9@j*N@)s^&ULq8PS?wg0_|@ zaBmY(RxG%3U0FlJASg(Ec_0TciQN77tAH;VZpapifNn9rNVz@YXWZQNE??=*n>W!L zpZpg41T)MX90t4Ng?*=@dpiPdnuLX&Ru~J4h%EGHsQljF_g(#VO@w3w6aPYHQuq^TkGK*AWWE#&nETb(BZ_=~&E!oFBg5u3u10%v9#l z;GXrDXPc@SD(1!w&v%^Lv2$~C*H)GGO5Fl3BK7p-)ul%=lF5r2ehbW}r>BtZR{A|g z*MI;lP#;F9)GXsJ`{n2hDW5Mo=8p$*f6yAd!gjZgB3EY`0~S-SPeom88iRSHWekuy zFRIsJKHui@wl>{^Bm(ClN6xRUP1XuIsB-R#LH`Z!is9RstbIE$U@Av;2Bgx$~p8cD=(k?>^mQaI1coYwbJ_|&9<_`obA(K zCrt^irvyBERs)FbJCzC;#kOh*ZfU6>5^{@nuvt)B%YzThKecCaWKrTEP;PR@DKIHk zy9lgrXlsG2km#u3Rub_%&q|5<;uU!LHmdpCN(SAC|)*Ue(2k`J8(EP+n7!O__U?vy{uIw$jr)$l&R|Bx{)TS z;`aEl^l*F{(b^Ea2rjzrj(?kB{VM)+yMr_0q~r3LPY%W+Rm1A8xw*N7a25Xh2(Uv{kviOabmE1R2(zgI^V{2FY!_Kz75`29$f&8e|m zRd@68QdUQ0WhF$Bh>a|-FRrXi_?o3&>(uc^b06E*rW+g*a{PPbfuK;PHTxDVT$xi9 zZZ$nL^sh_N*;#mZcXz4~22nJMCf5w8RIi2R>s(BF2U`>I(Z1*Tq18oM@X3=WrFeYK zr(D0EozaDd89(5GK|v}S8tFaUDyy3H_4RxAImc+-$Us_vV zM>yl=rbD&@<;Z;)gC>mUxhI!`7Zlli5ow{?4;t)>aX9xrc62VOwG}iqHI=O$cJtx1 zSrz>fK~d4giE3o%(cD{rU4`IO@#cWpaf#hso(GVf-oc17 ze?x)fUKgmuZnaXeJX;63wXADYvCBCLhQl&AbAM=V&J|}C5E9D&^eM`7XY#Yy3jNQhoz>&>G^_0ZD=vnwIHHj1Zmu!otY^Flu)3bGnBOL zXNhrFE3OYG*Hkz>!9Qr(r@-|i?;q4Twh-yXAdI#it7~g(R?x7k5}U1k)=Tix6fz2^ zJlkQHNxsY-|28)_7k#sr|_Lx-fG35p+xt`78Fu(0LW9XLx}42M;(U?o?ab`QSfb+iTvkdcjW{7E?J}&h|e(uyK4mUNOVbC`|9K?{R9+#lYU`6-(z@V=2SO z^7% z3-W7F=kwmU0%7_uo%20ci*{pRYk~NE4=B#%6jAn2N~@@O9T$XNhI^oQd!;Nxow<)0 zj`)Vy%1gY7-c#u}C9ERq(POWJ&Cv1+~zo+8WFni?xmyl7UWeTiV9~+dlH%-%*<(oKfUr z(!9=58)pn?E4-X967kn(?vs-v$4TzEZA&5cgmbC-?2J*3XFrM*sUBINB)#Wsd^qFD z;J7>!@y;`34OdcA1atqSIZ&m|G;xz|1vjIA9dhc@zmo<&15JfAmzS)u#7Y9aRKlei1|h1?VD6^v!xw;i1oq?LE7R+MI|C?`2?xoB60*`9I9 zXP^1am%a)_0)cz^p)+{w3HLT+I9b>~j`BAR{c!B${cwGAHniwna93YpwX*w?D6*J3 z+>-sc=2pb-N$lp#*}WqHko(*6^9V{qCwOS)1IKZrC)$hwOKn z4Y0TzO1FZUsLepx`TMi)NsUQLMT!+!yvHm^8ZrB$-d|IaBz(Rjx6F7%jbPPOTm3@1 z906hXj8UfK3@@i*G;PE?I^VBoBo&qw1g#5=bKg@JyGN%r`{}Q+dGya711xBoJ{X1@ zefvP?^f&M7>?UFDL{_}sa){G_zf zB(1^Dh^}lp#>mjC&X*$F1I$uomlgV!gpw*NuagA)Wvf*EKE`v$3SGIqmToPei2ygA zv%`RtT74EOFM~)?&GbwhcIP|c5xMtDlr<)a<-zY?fB#EYam9YVvs7ad_`?J;)Hl_u IyyN)dKj_G1-~a#s literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/warning-status-20x19.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/warning-status-20x19.png new file mode 100644 index 0000000000000000000000000000000000000000..0aa770cca4b8e749fcf928e3afea9f4e3fa5a003 GIT binary patch literal 805 zcmV+=1KRwFP)bD86c;TYM)TGyvCyhnJd1=(?b6x&gUe@7K zQPDNQs^B=#+iOTBxwCE^W514jUI@4bHc}|)^zyRuJ+^V9sdRU<)%RIyRB8aGCOt+k zJ6GSm)9F$v46_!>D`8-v5F~OreV0ce4!3VrDHfG`cU@E=u&`^l!EwSs-=}Oq{%d#CYKrT^n|}lH z`2Z1k{mSS1wF>b#h}0!$w^)q4@jT76JE~P!c#RwYt9;8-?eXPTeG;h_$)8G8bF8r6OS z@(tLP2CNC3pR@dUxPxrQ9tTGFKC1vbvl%<_(~(&CgI(2Q5-KRMiP)~8iIJp{KvGjv0v4o&!fxxfdzstuLrcrD%g$8b{W$xc zv(LQe_g>BjDJ6H?Z|hf5%pw(CwZKAP0Z!0VR9 z62k!72HTcc7L=3-0DL|Oh4nbl4jcncMk9vp%L0Orgw3Hqp*b$8wZTAHespxv!w%Het%d%urBIWc$@6!ac9uWkCbO0Np5o2&# zFrl!n0-q<7Y`=Na!Ev~6^ZNzW)uLb1Sa&Z?<-vr)x-KQhZ{3n>Mn)tL(lkv_T`kgn zKWn2Aqa*Kl?|vX9k%0m4AdUm+6l62*zLq5!7;plq6yJry`hub`p|HMvXh?2Or`WJJE@R?uu4nws>yh}>!G*ZZigRj_UEK&g}r*fy^M zD50?a*w`5J-4@N7C1`xpSJbC8C}`R1r=mh&S>DB*N=?9+CQn(Gdj`GyiiV=NZ(dm` z*tb7Gb+vavr_*u~fMOUd^m4Y#mwRsT%q$n|`pEAYCYu>&P)t)+xjCyU%$=7Xmf?hC zPjiMtUmJP9&7G&v)TDa?vTPX$g>_xAEL^il_vNQ-@?Xi$4>NRk+gvzbSg~tf(iBD4yy)=5NwZ+5Zg^7hbX2WTV?Z;zn@c+?VzMY9QJZl2U`tp+;C`UP%*N_>k`0;6Q=9%4)lKCW}cmd znK@Im0sU^PMn@%|ev+lKQn2hfwO|i2vasZ7mA<~h8CGCm|176#AN|%aG)?UBlvS4< z-hVH{iWM4j<|uhWes;M^dAVRubH+=->#M91Ulkn4Us$2VfakrI{rY;9{(gti(WyOZ zY7~YOl1$dU-)Wjyefnp!z2FS&_lu3UbKZ;f^w>Ed0E2^$ClFxQogf9TDyv@9me@9D zfB+BAj>S5y4X#&}Urt-yio#kT{;)uTL9wOawYbp3kM+eJio#N$?*W7Ch;>>UPW@=6 zJ;SKM88io%_Gy|}3>5PiVvSf^DqWAE9;6upGz8Oug@8IQjUGyq4uw2iwp4s6pj z@kFfCYQOJKA^=V%P4?yub_f3sI1VoXjld$HX5vE;fDN3R7)jcJqp?ow(saoG16=AQ UrO7$QK>z>%07*qoM6N<$f=g_KjQ{`u literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/warning-status-70x68.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/warning-status-70x68.png new file mode 100644 index 0000000000000000000000000000000000000000..7b4def1da8e289fefb84d9fd4d11fb62ca718741 GIT binary patch literal 4938 zcmV-Q6SeG#P)00006VoOIv0RI60 z0RN!9r;`8x010qNS#tmY3ljhU3ljkVnw%H_000McNliru;0yv1EH^7hEf)X)64yyY zK~#9!&765`9M^rvKkv=#E|*;1qC`>FX9Hh=VkB zkTy*k^v|LfiULL2*fCPHY0;)_f+R@N_(~$#mF(KFB|EZY-8V&16t6w!c<#YkuYnPiDPpMS6kdA`FPI4Cn%~ zKmq6nBH#ir1&C4(j%m#i-{Y|FtMkB=)|8+6Mi{h$K`miS z7|?5sJP6z;!rj)gIts~$q2!~8yfNfvgCQv*vqrj0&6_Hv6iP`7Qa+Pmz3;O!n^kvb zvP_6(SVRs2d#qIlfHUtoW3DG^pM4_RW{mWRup*Ax8XIWuc7_=m5DVWZ4h zGGm_U{vVYHB7gbUZR*5vyH11;itsUG*&fvB4Fc&3LsS$wR!9PbmAYwMbB5I7`>g=6 zP&H7ifz}Go6F-|}Lq4yTWpjKGkK@*|SA-t`r(4RH8KxvteE;|N%)Ixx&u2SpRk=Af z+-EI2t>GhaOmDR+)M^lgLTu`@e7&1)WX#qja4hmOiuQIACaH1H=QXYQZpPU>lh-ln9)i=uEi_?Oe~a2 z;CYHnMtX8N?&&C~6~GwaD|JtxeI<0lA!dS9|=I@3C8J4sF`(UHbAL)?_Kn(M$Q_ zsY&TKhR2QN_ktQ9FPCLYv4}301Zy2beqErXEJ`=AP^$@3Q!qU(>*I*eSj!V4eBjh6 z+yB+C<(72M{`FM*@4nuqjvlq$rJ~#&8y<~99;nr1aF&*c)~IFyN!_@$?&eZc)cVFv z%nD#!ydzAvjp)B}Eo3nF4J)#PaA;g~fR^HpU zH@-VId?u*zaIq-sgFyT=P;16?Qid4g{%Q@mtOG|eAG6$IX zsysa{eb#bc6!KsgvcZ`TbA+~0ro^#3zN-t`@|r?Hv2sYUX_I?hEDGn(3lrmp>1pR> zniXa%V4UYHWV3Q7sH&ggjn92PJ5ea8$-nxGz|6Ot1sPMVGH5N2grV%L)#!{PnG>Hh zO6wqSGuPd%SieDY+g6Xwn>9m2nnEYks&L_gB?v5;OkI#$0ghviP%0@ej%8h;pdK|b zQY^}DVEih^?0bCI3}e3br#Zh|l4Tb!%56a)TcU_=6T3X8Sxt!|$mO87SF?JxV#nN=ty5&73$AYN%zooZMHgP|})1DKOXcO5@|wZ4Dm+ zK2)vJ8-_9`iEET8Q3SQBFfgEbXqV5n?V6Q?nx0-Y14{bYwl)V~8k4ko7z@fsJ_yuy zYh@yvRnLF>U-Kt^_tTYVwu|*=qf{-NzLfm^-L-Zgdm^*+#d#V zf2AVbSEU4B{^M-+$uH(sTgxqB$l5SO)5L69HzW8ytX|`oa>t#T4I4Ct!s5plIu!#0 zirzj~xpQTV*5+2Ri;g>@NOsg}+)=I4J-hDYJNHbdF>n!Mxr#C6inLd2 zY%_+f)=GP9>cyoSHXrMl(%Yk0vsSa?4(EB6!jw8Nid$~=*tp4~qoeM&Tge)g)NI+JS+-1($+-P-+&XYl*2IwzhO{fihSR6*`hWR$n4fXiVMsxQo8p)uT9BV~ zcEWA9X>Qx9>FT=ilWFZlc+FZD1V%AjI4_h+Gez;1lsF`i)AhcL#q_Vc$`0P zQ3^&!Ewx(9Lz|okLqWhwrN{y3Vv$wAGO>78XxK|z>$GL!>{z}+@z`S- zKJl2(&?-Hrq|g%05i(IkUl>Y%5J(2V4+0rfiax+^)I}TPwpG^t8-G z$!$4Bdk5Ea@9phy^Ua!S1r8px6sH9%*KY6Lh}*VS5K$c|V|-)CTT5=K9fRu)PeW&6 z>7qM1DZKiM;qqmnr$;d~q}aOEqpSP+#|FOx$Yd1RC1^{n23N*}BS$Q6zhxL36Z-lUA|AJHltnM4vs1BUi>6i+E)HA9 z#_H9@M&lU-6li8KMu{{IN?OcLx225nE5h#GF~54-aPgwd#3n;SvZVN=z1`(f)6>GM zuPVe!)7~ioM|oQL31hG#*k<*m-c!Ezj2q)L<-`fgI|nVtj#QXoQ1gCR^RUdBKe);kFcjbahrddsqZzhQLL&a`$# zpjZ?}Mx1$_K0TK?*J4913(J=)mM?etSlW(hW>_NufUh-GpaxB{#=06u?K-XVQc*a1 z#BlV8WqR7tKdscoFa(ueckrNPa?&M?+qSzdLK+jzHx;d2 zw4HBPXf3UXf=ek5zUQe4kYdxoD3{<(Ob8R>g1@}(wvB;u3C6AnlM_O@1i3b5$_Yw= zRuD!|t_X4L8f_xbmUCHlTieylNsCt*ziyI^S`biHFs0NCFkDuODZn%;T&YJ2L*dF5 zVeE>~)2nD}tGnlm7cG}YEMX{UZ(-p}1G_ke6DKUynyc?@n-sQg^A;?2&F^3|BESk_ zrI-XJAu**~j$skjh!8fzmXt9Uhb=2sDAul3Gv3L>&Wt7}>&r$Ka_wo^!Wbvi7tYxk z;0IT_PUV8FnV{ZJZ*YVtMwQ}n3XJ~JALUL*5hLORxM5(lhDt>^dsaAm))LHGRBFYs zrdH#+UTZc3b2*4&ICs`^?5O3!dCTzd$YfLipm%I7 z8Ea$$xY7(Tp64!lY|Jt;B8*?D8@x_h7P7>=COx0EP%6UZ%a-9`;qsV^duHyY<}__; z%IQ)Qt(HJ*bvBdXY$oFflAfxf;EaN^K+vep7z5R+i)_!Fv0S{Eu6F_7bGg)w%&OMT zCC4!g4_hvrmpQ;xs=}3VC-!mNRLL{Ynp3$PC)?Xqb*AQvqO26h5jG%n3p89b5OI2P z^r)qypePhPG8yRVQgn8?K2drTi_LKQ8OI#w!rkH`MX?B%Mx1U{s_B~Fh8Z&@!s&ql zeXg(HnUe1cL8<~A3PV{fhAlvMqpb0LD3^q{-!h1}!qu8JiWP&di6-K_S0E~O}!q}la&V&@41CANX z<#gvS9@y+O=6m0btIvEZf6Q7rEK69Qwp1~mm=IomDQ4fkm>?7eS18u6*Q{FQg1;cR zcC|lUc~z@9Q5_o6tQb^S<4owt5zF!8mRe2l{f4`yMWuLMd%W)Zl;^PjpOP2LdW#~K z3ku%Fp%^s_OtmVUJSn{Mj^*9MuAsDXNV9RHrn|cy87~yET1lJyivE6=DCXM~XU?v3f|S8Us!CWx-QU@cc=({b!-$ZOYXZoWmc>Lx`P348a(y!?`3YD&oFn(Jo^ zjEHdH&bz#Kp8aV&hcSNl9e2VD``tgjnzoiVOe{UXR-msLaME(bF{<*UV_nt60*_Cs80 zTy*k;<;-cJQWZu=>nk~L5R};!v}vt)9>udt)z{zUmnSbqLGOHFKhJMp!NZSu%H52@ zSWWM6syGnQf!#xIogrN>z7mb8wJ3dfEbPM)w-D$=N?q(?-+VHA6F zIkmsNy{;`i+jh{iZAaZ*1AJw4nf#nfN!d_MSSvpeOUVi=nlY*Q?8#=`)QyfxvM4Bd zYGr|1OS(Q+q%Ouy#w?X&vR_1a>)D^iBa5N9;CT2EZ-+H}R^U-!4NXME^P}T58C>iE zjoDAdwV>np4)9&zXV&uO%P*NS!1Fr>0r<$qX$O;$Yj-}IF?tf!2rF$eZUZ|=ly;gsTGuQ4~za1 z*sVSO_vb$|V*uapavxZTDNANd65ojLt8t}xT`T@x!8b{s+*o9`eqAF3UIYFUcdb9n zX4Oo*;uqYuWR00Wp1ZBzG3U_S;@+-Z-tAWSH1Hu{b)q}@_X!}=z#-r{U=Q%xp4}!& znc^3q{igK*Jo4HpmSjpdFpg=((;~bo@L}M7V8{Cam{Y*>NpN}qIG2Lbs6AeHJh#SYa3No4{`X>u|sD(flF!e@<1EzM|3i{)mu9G>n(5{Cb@Bw2zs0p>1)Y0odlf6fj58y znT&ep#TVu-JbQL?@1ox4Jz>m3sXVR?vnO%o1Bp1V12!Zsz8C0Bz-q_++R*waX! zDFJ3OVaG|}7%&V>;C`8?1eE53(kh?d&{%9ncJ1;qR@e;OmYCPg35$9YK)QgQghjK* zbW%l}Od{9IiMjp?coR6aXScZuh#w7tZfhgc)pJ$ubED(`0O#eV1ha5h&Hw-a07*qo IM6N<$f{0O#X8-^I literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/warning-status-icon-20x19.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/warning-status-icon-20x19.png new file mode 100644 index 0000000000000000000000000000000000000000..7ec3b56dead2eddb86c2fc6642c23efeb60fdba0 GIT binary patch literal 665 zcmV;K0%rY*P)vxQHG1c9&p9o0oP5lr=67`--}5=&ALk(`qxAIc;~1~Y zsBDWt9TrFzAst}_CqN`a&4WJ|KHebQGPG`;C^m|d>FNH6QSLMCE%xYq8qg)1F(Ca| z{wTr<-WPWD!%)+s!X{u5wlLM-r*kvUbzzb1$f`Qx3cIErN=vON0&*BJ(Bm*)xl&fJ@((gk#qoiBmjT^{ho zSUfSZc7)E`ZesDoC17|Lu)9Ur8Zg!|j+5*n7Ekm7ecJ@NKw~VP7)-stu=#@pNWD8p z?U~m&$D1 zN^b%{=}m-z9|GgR4d8p%tEBDxfoj7U7dVH+qbB*sl4KS!^XpW#(AE}IZY#>#QUi6` z^(r?v3iAb|VBxzhup8Z)c@zah# z0K`P3^V6jB4?aYa-@?Ezwe3t8_<4};O!V-GiJ#C-DP>~~yg7#+IO=+pi8h>(N;*H? zbr1lNm}KsOBkA*R!@%FOrZZvSe`OYC#!S*^tN2g1!(wg`j7?}~}(rzCy84?v4q*h`=Fl0l1!-2THvP8j%kjieTRmjT4l zz$~wUT<>B-qx8&cz%mVxQ!012(+XL#Xzid{WXPfHsb(YXWUbM0YA+AX+e@ikpFH#f zm?K$>Y$UZ?nyUQZ$oA5$DZSWUI9L1lzTDkjx;3?wP1@de6S&_G5~cIsw50an{_Bxa z_W3@r+|#yuWyQ9$K6&sL;85>eu@CPUnHXQOAy?MslP~vt4|MjwAS;e4^}d3Pk0$e%|)rZw>64NEiMdLDC0-HhKGS%s ztj*+{p?gZ(`+*DH*fqf6wLsRIwL#ff&hA-lzi|rifNRaz$f6qXrPB60#d3AL8%H8w z7BF$xY@W?Wz%gJSu(Nqk1dsy1H(Qc%;CiuK{VNOk3nzhFCvd@kWB>pF07*qoM6N<$ Eg3&L&H~;_u literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/warning-status-icon-70x68.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/warning-status-icon-70x68.png new file mode 100644 index 0000000000000000000000000000000000000000..da206305618863f281b45bfe2176dfcd09549b6c GIT binary patch literal 3533 zcmV;;4KnhHP)00006VoOIv0RI60 z0RN!9r;`8x010qNS#tmY3ljhU3ljkVnw%H_000McNliru;0y=~0x-j@qci{j4O&S= zK~#9!<(vDDRo8XLKWm?J=gte}!DBEO7l;Y9G2q0GF_?#~+L2OK?GO2->f?vhs??}e zTe<0PtrRK!2U;a*s;aFjHEG(WRT4$5UyUKcK#U*Qn87yKW5CcE9`m02I%n^1e>msP z17=`6%mo5VjOO0MfwT8_f7f@dwf8#4BG~1(p9NsqwttxWp5uQW;m5i_D?uxO<-k&) z31|nF0SRyp7y$%~f%Su(U$*r-1J`?g-L5r``IO6{r9uGgt23^6kn(7V3Hyv&c5k z_hV!1zgGx71Z+ZV0PX{=Mp}x36Fx#d0!1j82o8l9Ot?4!=Rp`2usDUdf<@y}xB&D4 z9|9+(a0>W{2AC}dQwEPc7l}#-N>`w85BLqpZP6qY~$d89!0IRKjcZWve$ zW*{&#uqeM40j>nrXYdhbC&9gs#V5?gpMZ^ko9N#1t?FVhXi*p=Rk|ST0qrpKh~(EQ zR9h5kErg^+N{h&6V4(WH%KuD#7nXrjzhd>ZGcX4hwU`ywV}*Ngg-6Ua4MB8X(z~Gj zprgR)hBM|`qIUV%A2l0kCxjk_bTdl#N?|+rt%TI18VWTCH53xS=M1q~o<4mTn)M?& zd&~-$YXaAVSpi%jh3Eq__cMr&;No}9noeW!Q1?TBU0Wz~;!g>p0ERyJriIG!bwZDu z=}X|ZD3rV8D@zq>hSF*7qMRvYtNv-$$FBIIfJl%}F<(I$i(+$48;Dw$n`?d?MW?{O zD)d9(<3(l66jMTu@IOuI$-95DM1JBkDBS^muN1zBk}mlY3Kit5WNwqM^1_4PT#j4! z-~9h~m}|nKR?IbnD-2@sYew_~bA`j;J~_SRZ%007_NkZ?&wC)i;PL0$2$d%xJc*Ls z3YE`>w6qk}&~y!H2q0IirA>`bN>Dn6Ss7fR(_HZj7A@(4_+g>%-*^1EH?z=x!;JaL z45kdf|CcT#!(H;FhmrqvskVnu-6~&k!H+{orZJ=uV9aCj>I||Nb1k^I9dpI^u=qc) zc%PZ;we~FoT^s&b^I?vg&tpR}v<#(R2LJo=rC(J#_CQERGEn660o^K4^7XAhUmk@r zAsHqlBkKvZ-!sBjL63&gsbz!je|=taj<2QKV`u)}lCk3*N=F|->C+0;-3sMZ3KbNp zAg?^x`1Dh$&qY_yQ8@t?CYK@;e-EgrP*@|h+pKU7$;+3I0GFoEb2jG0*NQ&L*t;Es zIh6)LL;Riu<@8Q6kAiJWCYYBpd3_@D)VGgBQD5<)Tzu6db@mwwsIbwPBw z4$O5Ka|L1o7&-ghC>{LiXOZ7asO*+6ugzGJ7$`=XdPd6$Ng9|At@2<~h#jph z1>!hD3qvP2k3tF%D>U5;>@e5#&fu{>s9@2D-4A_hBD?Rv{G!re@7G%W$ZMNWvRmoI zc7^IjCPA6EG|dW16d9xADJT1e85*e&MTX0xSdj2ubagbdaYGkf9WmZ>1K{MSD<$|* zOvydYb^fNwFGKoA;8aG4_h$#@N>x@e+znxeLUoTqWrL=>&kYG|mhic#jlz<9D7qrf z31`ZIF+wm3ZB<+}3aU<(1UD4k1B4o2?S#tbgq5-Qjltu8I%M&Z;jT^J@K<`y?0HNm zuR{J;2(`Tml_d(bMF0g9JgKkJwsbvP*Z&sQ@<29urTH9Uup%m-K$pKqTImHVX4h{x z_YqP`NXH^`wRIMEJZ-8;>Bv4{@G8b!e*4);#teV(^;qfHa$h<3AinZ|RO^t^sgQi5 zz(~laG#A_HT(cDl_s)J}Qeb%e$5bX#ENa3QB@4()3SR-V+(PpX`H69Gy>(!wd(I58 zQ0-EOd}$Z3EBI1Z@TDw7ELM(USxAxsKCR3xnQ0}fwH6liuJ6vlw?)PzOr`o z%nu4z>R3D-rI3yS|RXL8`UN+y*~!YJKVScUmShUxivbgnJdz)if{~+@4zFk*vj}^-3q! zW0iiO-uBgDANtcd(- zh1xwpF)PRY$QrQaQi!rhbT^s`DD5Uxmw^>BjS1CNKsP8}j6(4j)C!Vc21$3lVvQAQ zD}l~MC}Dq*8RJk`D$?#r#z;vAs11njPEhhqW;_`VlZ;7Qq)=S&w%nb<83m9c%tcv( zr_hw^i`~VrWZMx0SI_NY2zGaaG94VuEORB7)!=t+oSp@oY{$w0RUp0lQr@|=Cavbc zSedM+?WVD2vU6(A8{!!aL^si@$R`;v=Eg9}dAzzwbTuq0&W%e{WZHq~&9rea4-lKV zA;H+q0H)3qg%as_FU~CMsmc3k%p%M+laBYIP{M8=a{|}|SsQH~n8HP1B-C9bk8au7 z^*n$v>mzFU$Sf*P1IA4Nryy*|`}`XN%q=~;zNU?UK8~7u&bp=*)k$D1DwdFXrDQGA z135Sisqm+tT}Or4Z)@z?_0^99T~J6q>E8V9k*P2NSkzXssO5b#cOsAF4NmcH?b-EA zZLvPElPJAdzj*QW-^duV;?e&qOFDVs@Kg7KE^*hGjbZMHnLC0sF(a|)`_FnPH9?Hg|77 zzT&axXD}vi>evFD$XOGQg5H3nQ(`mF-H=+zmq^BXb3K_Et}QB(jP)Yl&|0Ym^no1( z`j%|`MhbB5rEI!uo*A6kgSX=Gmr9b}xo~K2JFpvAL-yX{;uI`-k?5=al+OJN7PZ!; zcnUZS`4Ya`pS?7(c(u~>v!PMUy=b`SF`Mkab4{!29!9+U^Z(9;Lr($N${nxB%osTM z#D`$7lD)+F$l@`l&X@_3@#A?Xs9iOFvp5GOUQBnYKPUlC81||~#kb7dq_z~=AIv}X z&S1>s2+TDPfqp2Z<0$kju1g8bVXjE_0_|Kb=Nh3gCEy6~ve4TrAA6yHt{Y>3l{;RU zadGA2|62n-960n;8)zG-1-NhVGUEI#k((c%`+$AGenNV-e(}QLrvO;~$Pbv2;9a8} zK7+aMf&Clv=)b>>Eo5~duLCauhkKqlI58i;%aHr|7c<6ee{u5Kz-zzOf*;+1!dG)a zUPc36OKvbIr-6gO%fOF%b{-f47&y4g0QKxVFw4%-b*$&HpOlgHg8eht_ksS~9EdB0 z>5~i;`&TUfiN)ePiCO(*q38o9Ka=qA@x8Pume~PwB3$*e*Sxc z>}Ao{fqhu?^PV01E&=@CEB9yfn}sQ}0y8ITcD<;9SDzWg{1MD9BOL?%4zTSSxx%fq z8t_^!v@ZiEEGmvnrBiM5Mt4>lIJhTA_GdG%AKJ4Gh2I8t0c&&JX}L`R83lTQ7l8f1 zYdt#;Bz2}()P`$boq5P(|8rxEnFY_2nEf5%Fz6XzC$RN40HzOk5qK3i3Y@KjvQU4# zrVSk0n`_Sg8P_iy+S@7lr+_Db$AEP}YyNXH7*qkq$=)96&%t;ec&lgUf&MAx&mDah znK6?o@+k8A@+$Nt;2~gp&Yq1lw&y09mYcklfnt52!5l2+6Mb%oF;`}YY54*#{Mm87 z^5%3s+EoPZ&m-2h{BuvvoED%tuK@~V-CHjI9nCThpHJN9lS;GQHY)kgXr6GM0^R_Q zTGVuW<@TSZGc|J8b)!#X7&B`r&*fb7+JF_gIIjaX)C81v+yUO}Ei} z7jZbxTrcMCdJK3I= JList, ServiceLoader} import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import org.apache.spark.{JobExecutionStatus, SecurityManager, SparkConf, SparkContext} import org.apache.spark.internal.Logging @@ -36,7 +37,7 @@ import org.apache.spark.util.Utils /** * Top level user interface for a Spark application. */ -private[spark] class SparkUI private ( +class SparkUI private ( val store: AppStatusStore, val sc: Option[SparkContext], val conf: SparkConf, @@ -66,7 +67,7 @@ private[spark] class SparkUI private ( attachTab(new EnvironmentTab(this, store)) attachTab(new ExecutorsTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) - attachHandler(createRedirectHandler("/", "/jobs/", basePath = basePath)) + // attachHandler(createRedirectHandler("/", "/jobs/", basePath = basePath)) attachHandler(ApiRootResource.getServletHandler(this)) // These should be POST only, but, the YARN AM proxy won't proxy POSTs @@ -95,6 +96,10 @@ private[spark] class SparkUI private ( appId = id } + def setTabs(newTabs: ArrayBuffer[WebUITab]): Unit = { + tabs = newTabs + } + /** Stop the server behind this web interface. Only valid after bind(). */ override def stop() { super.stop() diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index ba798df13c95d..a80ace578f5a3 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -165,6 +165,7 @@ private[spark] object UIUtils extends Logging { + @@ -231,8 +232,8 @@ private[spark] object UIUtils extends Logging {

@@ -295,6 +296,7 @@ private[spark] object UIUtils extends Logging { + {getProductDocLinkNode()} @@ -599,11 +601,18 @@ private[spark] object UIUtils extends Logging { } } - def getProductVersionNode = { + def getProductVersionNode(): Node = { val versionTooltipText = "SnappyData Ver. " + SparkUI.getProductVersion + " ( Underlying Spark Ver. " + org.apache.spark.SPARK_VERSION + " )" {SparkUI.getProductVersion} } + + def getProductDocLinkNode(): Node = { + + } + } From 587a3f9aa4788ca793df415b38c3605f45a9d58e Mon Sep 17 00:00:00 2001 From: Amogh Shetkar Date: Wed, 21 Dec 2016 14:51:24 +0530 Subject: [PATCH 048/142] [SNAPPYDATA] Spark Version 2.0.3-2 --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 402593595ce2a..2ab964fbd6eb4 100644 --- a/build.gradle +++ b/build.gradle @@ -45,7 +45,7 @@ allprojects { apply plugin: 'idea' group = 'io.snappydata' - version = '2.0.3-1' + version = '2.0.3-2' ext { scalaBinaryVersion = '2.11' From f49a3ac3f7a0bdc9660b7023384383d6c1f531ef Mon Sep 17 00:00:00 2001 From: snappy-sachin Date: Tue, 3 Jan 2017 15:36:57 +0530 Subject: [PATCH 049/142] SNAP-1281: UI does not show up if spark shell is run without snappydata (#42) Fixes: Re-enabling the default spark redirection handler to redirect user to spark jobs page. --- core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index c25bffa83b496..a4f8f2b798b01 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -67,7 +67,7 @@ class SparkUI private ( attachTab(new EnvironmentTab(this, store)) attachTab(new ExecutorsTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) - // attachHandler(createRedirectHandler("/", "/jobs/", basePath = basePath)) + attachHandler(createRedirectHandler("/", "/jobs/", basePath = basePath)) attachHandler(ApiRootResource.getServletHandler(this)) // These should be POST only, but, the YARN AM proxy won't proxy POSTs From 9f91917eddc623c85a5510e180eb3b3ed82308f9 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Wed, 30 Nov 2016 18:19:50 +0530 Subject: [PATCH 050/142] [SNAP-1185] Guard logging and time measurements (#28) - add explicit log-level check for some log lines in java code (scala code already uses logging arguments as pass-by-name) - for System.currentTimeInMillis() calls that are used only by logging, guard it with the appropriate log-level check - use System.nanoTime in a few places where duration is to be measured; also using a DoubleAccumulator to add results for better accuracy - cache commonly used logging.is*Enabled flags - use explicit flag variable in Logging initialized lazily instead of lazy val that causes hang in streaming tests for some reason even if marked transient - renamed flags for consistency - add handling for possible DoubleAccumulators in a couple of places that expect only LongAccumulators in TaskMetrics - fixing scalastyle error due to 2c432045 Conflicts: core/src/main/scala/org/apache/spark/executor/Executor.scala core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala core/src/main/scala/org/apache/spark/storage/BlockManager.scala Conflicts: common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java core/src/main/scala/org/apache/spark/MapOutputTracker.scala core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala core/src/main/scala/org/apache/spark/executor/Executor.scala core/src/main/scala/org/apache/spark/internal/Logging.scala core/src/main/scala/org/apache/spark/storage/BlockManager.scala core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala --- .../spark/network/client/TransportClient.java | 27 +- .../network/protocol/MessageDecoder.java | 4 +- .../server/TransportRequestHandler.java | 4 +- .../org/apache/spark/MapOutputTracker.scala | 4 +- .../spark/broadcast/TorrentBroadcast.scala | 5 +- .../org/apache/spark/executor/Executor.scala | 27 +- .../spark/executor/ShuffleReadMetrics.scala | 16 +- .../apache/spark/executor/TaskMetrics.scala | 25 +- .../org/apache/spark/internal/Logging.scala | 63 ++- .../apache/spark/rpc/RpcEndpointAddress.scala | 2 +- .../apache/spark/scheduler/ResultTask.scala | 4 +- .../spark/scheduler/ShuffleMapTask.scala | 4 +- .../org/apache/spark/scheduler/Task.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 364 +++++++++--------- .../org/apache/spark/storage/DiskStore.scala | 5 +- .../storage/ShuffleBlockFetcherIterator.scala | 20 +- .../org/apache/spark/util/JsonProtocol.scala | 5 +- 17 files changed, 324 insertions(+), 257 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java index 8f354ad78bbaa..5e8df06207b16 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java @@ -131,9 +131,10 @@ public void setClientId(String id) { */ public void fetchChunk( long streamId, - int chunkIndex, - ChunkReceivedCallback callback) { - long startTime = System.currentTimeMillis(); + final int chunkIndex, + final ChunkReceivedCallback callback) { + final boolean isTraceEnabled = logger.isTraceEnabled(); + final long startTime = isTraceEnabled ? System.currentTimeMillis() : 0L; if (logger.isDebugEnabled()) { logger.debug("Sending fetch chunk request {} to {}", chunkIndex, getRemoteAddress(channel)); } @@ -144,7 +145,7 @@ public void fetchChunk( channel.writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener(future -> { if (future.isSuccess()) { long timeTaken = System.currentTimeMillis() - startTime; - if (logger.isTraceEnabled()) { + if (isTraceEnabled) { logger.trace("Sending request {} to {} took {} ms", streamChunkId, getRemoteAddress(channel), timeTaken); } @@ -169,8 +170,9 @@ public void fetchChunk( * @param streamId The stream to fetch. * @param callback Object to call with the stream data. */ - public void stream(String streamId, StreamCallback callback) { - long startTime = System.currentTimeMillis(); + public void stream(final String streamId, final StreamCallback callback) { + final boolean isTraceEnabled = logger.isTraceEnabled(); + final long startTime = isTraceEnabled ? System.currentTimeMillis() : 0L; if (logger.isDebugEnabled()) { logger.debug("Sending stream request for {} to {}", streamId, getRemoteAddress(channel)); } @@ -183,7 +185,7 @@ public void stream(String streamId, StreamCallback callback) { channel.writeAndFlush(new StreamRequest(streamId)).addListener(future -> { if (future.isSuccess()) { long timeTaken = System.currentTimeMillis() - startTime; - if (logger.isTraceEnabled()) { + if (isTraceEnabled) { logger.trace("Sending request for {} to {} took {} ms", streamId, getRemoteAddress(channel), timeTaken); } @@ -210,9 +212,10 @@ public void stream(String streamId, StreamCallback callback) { * @param callback Callback to handle the RPC's reply. * @return The RPC's id. */ - public long sendRpc(ByteBuffer message, RpcResponseCallback callback) { - long startTime = System.currentTimeMillis(); - if (logger.isTraceEnabled()) { + public long sendRpc(ByteBuffer message, final RpcResponseCallback callback) { + final boolean isTraceEnabled = logger.isTraceEnabled(); + final long startTime = isTraceEnabled ? System.currentTimeMillis() : 0L; + if (isTraceEnabled) { logger.trace("Sending RPC to {}", getRemoteAddress(channel)); } @@ -222,8 +225,8 @@ public long sendRpc(ByteBuffer message, RpcResponseCallback callback) { channel.writeAndFlush(new RpcRequest(requestId, new NioManagedBuffer(message))) .addListener(future -> { if (future.isSuccess()) { - long timeTaken = System.currentTimeMillis() - startTime; - if (logger.isTraceEnabled()) { + if (isTraceEnabled) { + long timeTaken = System.currentTimeMillis() - startTime; logger.trace("Sending request {} to {} took {} ms", requestId, getRemoteAddress(channel), timeTaken); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java index 39a7495828a8a..a8cb9726f728c 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java @@ -44,7 +44,9 @@ public void decode(ChannelHandlerContext ctx, ByteBuf in, List out) { Message.Type msgType = Message.Type.decode(in); Message decoded = decode(msgType, in); assert decoded.type() == msgType; - logger.trace("Received message {}: {}", msgType, decoded); + if (logger.isTraceEnabled()) { + logger.trace("Received message {}: {}", msgType, decoded); + } out.add(decoded); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java index e94453578e6b0..c1f28e98bff29 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -221,7 +221,9 @@ private ChannelFuture respond(Encodable result) { SocketAddress remoteAddress = channel.remoteAddress(); return channel.writeAndFlush(result).addListener(future -> { if (future.isSuccess()) { - logger.trace("Sent result {} to client {}", result, remoteAddress); + if (logger.isTraceEnabled()) { + logger.trace("Sent result {} to client {}", result, remoteAddress); + } } else { logger.error(String.format("Error sending result %s to %s; closing connection", result, remoteAddress), future.cause()); diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 195fd4f818b36..9c8d7d8797a49 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -693,7 +693,7 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them") - val startTime = System.currentTimeMillis + val startTime = if (isDebugEnabled) System.currentTimeMillis else 0L var fetchedStatuses: Array[MapStatus] = null fetching.synchronized { // Someone else is fetching it; wait for them to be done @@ -730,7 +730,7 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr } } } - logDebug(s"Fetching map output statuses for shuffle $shuffleId took " + + if (isDebugEnabled) logDebug(s"Fetching map output statuses for shuffle $shuffleId took " + s"${System.currentTimeMillis - startTime} ms") if (fetchedStatuses != null) { diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index e125095cf4777..c393c82f54e22 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -227,9 +227,10 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) } case None => logInfo("Started reading broadcast variable " + id) - val startTimeMs = System.currentTimeMillis() + val startTimeMs = if (isDebugEnabled) System.currentTimeMillis() else 0L val blocks = readBlocks() - logInfo("Reading broadcast variable " + id + " took" + Utils.getUsedTimeMs(startTimeMs)) + if (isDebugEnabled) logDebug("Reading broadcast variable " + id + " took" + + Utils.getUsedTimeMs(startTimeMs)) try { val obj = TorrentBroadcast.unBlockifyObject[T]( diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 4f1b35415e5af..80da4e1a6f1dc 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -307,7 +307,7 @@ private[spark] class Executor( Thread.currentThread.setName(threadName) val threadMXBean = ManagementFactory.getThreadMXBean val taskMemoryManager = new TaskMemoryManager(env.memoryManager, taskId) - val deserializeStartTime = System.currentTimeMillis() + val deserializeStartTime = System.nanoTime() val deserializeStartCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) { threadMXBean.getCurrentThreadCpuTime } else 0L @@ -351,7 +351,7 @@ private[spark] class Executor( } // Run the actual task and measure its runtime. - taskStart = System.currentTimeMillis() + taskStart = System.nanoTime() taskStartCpu = if (threadMXBean.isCurrentThreadCpuTimeSupported) { threadMXBean.getCurrentThreadCpuTime } else 0L @@ -395,7 +395,8 @@ private[spark] class Executor( s"unrecoverable fetch failures! Most likely this means user code is incorrectly " + s"swallowing Spark's internal ${classOf[FetchFailedException]}", fetchFailure) } - val taskFinish = System.currentTimeMillis() + + val taskFinish = System.nanoTime() val taskFinishCpu = if (threadMXBean.isCurrentThreadCpuTimeSupported) { threadMXBean.getCurrentThreadCpuTime } else 0L @@ -404,22 +405,24 @@ private[spark] class Executor( task.context.killTaskIfInterrupted() val resultSer = env.serializer.newInstance() - val beforeSerialization = System.currentTimeMillis() + val beforeSerialization = System.nanoTime() val valueBytes = resultSer.serialize(value) - val afterSerialization = System.currentTimeMillis() + val afterSerialization = System.nanoTime() // Deserialization happens in two parts: first, we deserialize a Task object, which // includes the Partition. Second, Task.run() deserializes the RDD and function to be run. - task.metrics.setExecutorDeserializeTime( - (taskStart - deserializeStartTime) + task.executorDeserializeTime) + task.metrics.setExecutorDeserializeTime(math.max( + taskStart - deserializeStartTime + task.executorDeserializeTime, 0L) / 1000000.0) task.metrics.setExecutorDeserializeCpuTime( (taskStartCpu - deserializeStartCpuTime) + task.executorDeserializeCpuTime) // We need to subtract Task.run()'s deserialization time to avoid double-counting - task.metrics.setExecutorRunTime((taskFinish - taskStart) - task.executorDeserializeTime) + task.metrics.setExecutorRunTime(math.max( + taskFinish - taskStart - task.executorDeserializeTime, 0L) / 1000000.0) task.metrics.setExecutorCpuTime( (taskFinishCpu - taskStartCpu) - task.executorDeserializeCpuTime) task.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime) - task.metrics.setResultSerializationTime(afterSerialization - beforeSerialization) + task.metrics.setResultSerializationTime(math.max( + afterSerialization - beforeSerialization, 0L) / 1000000.0) // Expose task metrics using the Dropwizard metrics system. // Update task metrics counters @@ -542,7 +545,8 @@ private[spark] class Executor( // Collect latest accumulator values to report back to the driver val accums: Seq[AccumulatorV2[_, _]] = if (task != null) { - task.metrics.setExecutorRunTime(System.currentTimeMillis() - taskStart) + task.metrics.setExecutorRunTime( + math.max(System.nanoTime() - taskStart, 0L) / 1000000.0) task.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime) task.collectAccumulatorUpdates(taskFailed = true) } else { @@ -748,7 +752,8 @@ private[spark] class Executor( * Download any missing dependencies if we receive a new set of files and JARs from the * SparkContext. Also adds any new JARs we fetched to the class loader. */ - protected def updateDependencies(newFiles: HashMap[String, Long], newJars: HashMap[String, Long]) { + protected def updateDependencies(newFiles: HashMap[String, Long], + newJars: HashMap[String, Long]) { lazy val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) synchronized { // Fetch missing dependencies diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala index 4be395c8358b2..5a435f2f9a1ba 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala @@ -18,7 +18,7 @@ package org.apache.spark.executor import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.util.LongAccumulator +import org.apache.spark.util.{DoubleAccumulator, LongAccumulator} /** @@ -33,7 +33,7 @@ class ShuffleReadMetrics private[spark] () extends Serializable { private[executor] val _remoteBytesRead = new LongAccumulator private[executor] val _remoteBytesReadToDisk = new LongAccumulator private[executor] val _localBytesRead = new LongAccumulator - private[executor] val _fetchWaitTime = new LongAccumulator + private[executor] val _fetchWaitTime = new DoubleAccumulator private[executor] val _recordsRead = new LongAccumulator /** @@ -66,7 +66,7 @@ class ShuffleReadMetrics private[spark] () extends Serializable { * blocking on shuffle input data. For instance if block B is being fetched while the task is * still not finished processing block A, it is not considered to be blocking on block B. */ - def fetchWaitTime: Long = _fetchWaitTime.sum + def fetchWaitTime: Long = _fetchWaitTime.sum.toLong /** * Total number of records read from the shuffle by this task. @@ -88,7 +88,7 @@ class ShuffleReadMetrics private[spark] () extends Serializable { private[spark] def incRemoteBytesRead(v: Long): Unit = _remoteBytesRead.add(v) private[spark] def incRemoteBytesReadToDisk(v: Long): Unit = _remoteBytesReadToDisk.add(v) private[spark] def incLocalBytesRead(v: Long): Unit = _localBytesRead.add(v) - private[spark] def incFetchWaitTime(v: Long): Unit = _fetchWaitTime.add(v) + private[spark] def incFetchWaitTime(v: Double): Unit = _fetchWaitTime.add(v) private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) private[spark] def setRemoteBlocksFetched(v: Int): Unit = _remoteBlocksFetched.setValue(v) @@ -96,7 +96,7 @@ class ShuffleReadMetrics private[spark] () extends Serializable { private[spark] def setRemoteBytesRead(v: Long): Unit = _remoteBytesRead.setValue(v) private[spark] def setRemoteBytesReadToDisk(v: Long): Unit = _remoteBytesReadToDisk.setValue(v) private[spark] def setLocalBytesRead(v: Long): Unit = _localBytesRead.setValue(v) - private[spark] def setFetchWaitTime(v: Long): Unit = _fetchWaitTime.setValue(v) + private[spark] def setFetchWaitTime(v: Double): Unit = _fetchWaitTime.setValue(v) private[spark] def setRecordsRead(v: Long): Unit = _recordsRead.setValue(v) /** @@ -134,7 +134,7 @@ private[spark] class TempShuffleReadMetrics { private[this] var _remoteBytesRead = 0L private[this] var _remoteBytesReadToDisk = 0L private[this] var _localBytesRead = 0L - private[this] var _fetchWaitTime = 0L + private[this] var _fetchWaitTime = 0.0 private[this] var _recordsRead = 0L def incRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched += v @@ -142,7 +142,7 @@ private[spark] class TempShuffleReadMetrics { def incRemoteBytesRead(v: Long): Unit = _remoteBytesRead += v def incRemoteBytesReadToDisk(v: Long): Unit = _remoteBytesReadToDisk += v def incLocalBytesRead(v: Long): Unit = _localBytesRead += v - def incFetchWaitTime(v: Long): Unit = _fetchWaitTime += v + def incFetchWaitTime(v: Double): Unit = _fetchWaitTime += v def incRecordsRead(v: Long): Unit = _recordsRead += v def remoteBlocksFetched: Long = _remoteBlocksFetched @@ -150,6 +150,6 @@ private[spark] class TempShuffleReadMetrics { def remoteBytesRead: Long = _remoteBytesRead def remoteBytesReadToDisk: Long = _remoteBytesReadToDisk def localBytesRead: Long = _localBytesRead - def fetchWaitTime: Long = _fetchWaitTime + def fetchWaitTime: Double = _fetchWaitTime def recordsRead: Long = _recordsRead } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 85b2745a2aec4..dc6dc6878f567 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -44,13 +44,13 @@ import org.apache.spark.util._ @DeveloperApi class TaskMetrics private[spark] () extends Serializable { // Each metric is internally represented as an accumulator - private val _executorDeserializeTime = new LongAccumulator + private val _executorDeserializeTime = new DoubleAccumulator private val _executorDeserializeCpuTime = new LongAccumulator - private val _executorRunTime = new LongAccumulator + private val _executorRunTime = new DoubleAccumulator private val _executorCpuTime = new LongAccumulator private val _resultSize = new LongAccumulator private val _jvmGCTime = new LongAccumulator - private val _resultSerializationTime = new LongAccumulator + private val _resultSerializationTime = new DoubleAccumulator private val _memoryBytesSpilled = new LongAccumulator private val _diskBytesSpilled = new LongAccumulator private val _peakExecutionMemory = new LongAccumulator @@ -59,7 +59,7 @@ class TaskMetrics private[spark] () extends Serializable { /** * Time taken on the executor to deserialize this task. */ - def executorDeserializeTime: Long = _executorDeserializeTime.sum + def executorDeserializeTime: Long = _executorDeserializeTime.sum.toLong /** * CPU Time taken on the executor to deserialize this task in nanoseconds. @@ -69,7 +69,7 @@ class TaskMetrics private[spark] () extends Serializable { /** * Time the executor spends actually running the task (including fetching shuffle data). */ - def executorRunTime: Long = _executorRunTime.sum + def executorRunTime: Long = _executorRunTime.sum.toLong /** * CPU Time the executor spends actually running the task @@ -90,7 +90,7 @@ class TaskMetrics private[spark] () extends Serializable { /** * Amount of time spent serializing the task result. */ - def resultSerializationTime: Long = _resultSerializationTime.sum + def resultSerializationTime: Long = _resultSerializationTime.sum.toLong /** * The number of in-memory bytes spilled by this task. @@ -126,15 +126,15 @@ class TaskMetrics private[spark] () extends Serializable { } // Setters and increment-ers - private[spark] def setExecutorDeserializeTime(v: Long): Unit = + private[spark] def setExecutorDeserializeTime(v: Double): Unit = _executorDeserializeTime.setValue(v) private[spark] def setExecutorDeserializeCpuTime(v: Long): Unit = _executorDeserializeCpuTime.setValue(v) - private[spark] def setExecutorRunTime(v: Long): Unit = _executorRunTime.setValue(v) + private[spark] def setExecutorRunTime(v: Double): Unit = _executorRunTime.setValue(v) private[spark] def setExecutorCpuTime(v: Long): Unit = _executorCpuTime.setValue(v) private[spark] def setResultSize(v: Long): Unit = _resultSize.setValue(v) private[spark] def setJvmGCTime(v: Long): Unit = _jvmGCTime.setValue(v) - private[spark] def setResultSerializationTime(v: Long): Unit = + private[spark] def setResultSerializationTime(v: Double): Unit = _resultSerializationTime.setValue(v) private[spark] def incMemoryBytesSpilled(v: Long): Unit = _memoryBytesSpilled.add(v) private[spark] def incDiskBytesSpilled(v: Long): Unit = _diskBytesSpilled.add(v) @@ -299,9 +299,10 @@ private[spark] object TaskMetrics extends Logging { if (name == UPDATED_BLOCK_STATUSES) { tm.setUpdatedBlockStatuses(value.asInstanceOf[java.util.List[(BlockId, BlockStatus)]]) } else { - tm.nameToAccums.get(name).foreach( - _.asInstanceOf[LongAccumulator].setValue(value.asInstanceOf[Long]) - ) + tm.nameToAccums.get(name).foreach { + case l: LongAccumulator => l.setValue(value.asInstanceOf[Long]) + case d => d.asInstanceOf[DoubleAccumulator].setValue(value.asInstanceOf[Double]) + } } } tm diff --git a/core/src/main/scala/org/apache/spark/internal/Logging.scala b/core/src/main/scala/org/apache/spark/internal/Logging.scala index c0d709ad25f29..a1105b5ce1a8f 100644 --- a/core/src/main/scala/org/apache/spark/internal/Logging.scala +++ b/core/src/main/scala/org/apache/spark/internal/Logging.scala @@ -34,6 +34,8 @@ trait Logging { // be serialized and used on another machine @transient private var log_ : Logger = null + @transient private[this] var levelFlags: Int = _ + // Method to get the logger name for this object protected def logName = { // Ignore trailing $'s in the class names for Scala objects @@ -49,17 +51,54 @@ trait Logging { log_ } + private def setLevel(value: Boolean, enabled: Int, disabled: Int): Unit = { + if (value) levelFlags |= enabled else levelFlags |= disabled + } + + protected final def isInfoEnabled: Boolean = { + val levelFlags = this.levelFlags + if ((levelFlags & Logging.INFO_ENABLED) != 0) true + else if ((levelFlags & Logging.INFO_DISABLED) != 0) false + else { + val value = log.isInfoEnabled + setLevel(value, Logging.INFO_ENABLED, Logging.INFO_DISABLED) + value + } + } + + protected final def isDebugEnabled: Boolean = { + val levelFlags = this.levelFlags + if ((levelFlags & Logging.DEBUG_DISABLED) != 0) false + else if ((levelFlags & Logging.DEBUG_ENABLED) != 0) true + else { + val value = log.isDebugEnabled + setLevel(value, Logging.DEBUG_ENABLED, Logging.DEBUG_DISABLED) + value + } + } + + protected final def isTraceEnabled: Boolean = { + val levelFlags = this.levelFlags + if ((levelFlags & Logging.TRACE_DISABLED) != 0) false + else if ((levelFlags & Logging.TRACE_ENABLED) != 0) true + else { + val value = log.isTraceEnabled + setLevel(value, Logging.TRACE_ENABLED, Logging.TRACE_DISABLED) + value + } + } + // Log methods that take only a String protected def logInfo(msg: => String) { - if (log.isInfoEnabled) log.info(msg) + if (isInfoEnabled) log.info(msg) } protected def logDebug(msg: => String) { - if (log.isDebugEnabled) log.debug(msg) + if (isDebugEnabled) log.debug(msg) } protected def logTrace(msg: => String) { - if (log.isTraceEnabled) log.trace(msg) + if (isTraceEnabled) log.trace(msg) } protected def logWarning(msg: => String) { @@ -72,15 +111,15 @@ trait Logging { // Log methods that take Throwables (Exceptions/Errors) too protected def logInfo(msg: => String, throwable: Throwable) { - if (log.isInfoEnabled) log.info(msg, throwable) + if (isInfoEnabled) log.info(msg, throwable) } protected def logDebug(msg: => String, throwable: Throwable) { - if (log.isDebugEnabled) log.debug(msg, throwable) + if (isDebugEnabled) log.debug(msg, throwable) } protected def logTrace(msg: => String, throwable: Throwable) { - if (log.isTraceEnabled) log.trace(msg, throwable) + if (isTraceEnabled) log.trace(msg, throwable) } protected def logWarning(msg: => String, throwable: Throwable) { @@ -91,10 +130,6 @@ trait Logging { if (log.isErrorEnabled) log.error(msg, throwable) } - protected def isTraceEnabled(): Boolean = { - log.isTraceEnabled - } - protected def initializeLogIfNecessary(isInterpreter: Boolean): Unit = { initializeLogIfNecessary(isInterpreter, silent = false) } @@ -163,6 +198,14 @@ trait Logging { } private[spark] object Logging { + + private val INFO_ENABLED = 0x1 + private val INFO_DISABLED = 0x2 + private val DEBUG_ENABLED = 0x4 + private val DEBUG_DISABLED = 0x8 + private val TRACE_ENABLED = 0x10 + private val TRACE_DISABLED = 0x20 + @volatile private var initialized = false @volatile private var defaultRootLevel: Level = null @volatile private var defaultSparkLog4jConfig = false diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointAddress.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointAddress.scala index fdbccc9e74c37..21061fb11a946 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointAddress.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointAddress.scala @@ -37,7 +37,7 @@ private[spark] case class RpcEndpointAddress(rpcAddress: RpcAddress, name: Strin this(RpcAddress(host, port), name) } - override val toString = if (rpcAddress != null) { + override def toString: String = if (rpcAddress != null) { s"spark://$name@${rpcAddress.host}:${rpcAddress.port}" } else { s"spark-client://$name" diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index e36c759a42556..2acac37dcff94 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -72,14 +72,14 @@ private[spark] class ResultTask[T, U]( override def runTask(context: TaskContext): U = { // Deserialize the RDD and the func using the broadcast variables. val threadMXBean = ManagementFactory.getThreadMXBean - val deserializeStartTime = System.currentTimeMillis() + val deserializeStartTime = System.nanoTime() val deserializeStartCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) { threadMXBean.getCurrentThreadCpuTime } else 0L val ser = SparkEnv.get.closureSerializer.newInstance() val (rdd, func) = ser.deserialize[(RDD[T], (TaskContext, Iterator[T]) => U)]( ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) - _executorDeserializeTime = System.currentTimeMillis() - deserializeStartTime + _executorDeserializeTime = math.max(System.nanoTime() - deserializeStartTime, 0L) _executorDeserializeCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) { threadMXBean.getCurrentThreadCpuTime - deserializeStartCpuTime } else 0L diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 7a25c47e2cab3..ef2b3e8764b71 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -77,14 +77,14 @@ private[spark] class ShuffleMapTask( override def runTask(context: TaskContext): MapStatus = { // Deserialize the RDD using the broadcast variable. val threadMXBean = ManagementFactory.getThreadMXBean - val deserializeStartTime = System.currentTimeMillis() + val deserializeStartTime = System.nanoTime() val deserializeStartCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) { threadMXBean.getCurrentThreadCpuTime } else 0L val ser = SparkEnv.get.closureSerializer.newInstance() val (rdd, dep) = ser.deserialize[(RDD[_], ShuffleDependency[_, _, _])]( ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) - _executorDeserializeTime = System.currentTimeMillis() - deserializeStartTime + _executorDeserializeTime = math.max(System.nanoTime() - deserializeStartTime, 0L) _executorDeserializeCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) { threadMXBean.getCurrentThreadCpuTime - deserializeStartCpuTime } else 0L diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index f536fc2a5f0a1..2a9b0628c1131 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -179,7 +179,7 @@ private[spark] abstract class Task[T]( def reasonIfKilled: Option[String] = Option(_reasonIfKilled) /** - * Returns the amount of time spent deserializing the RDD and function to be run. + * Returns the amount of time spent deserializing the RDD and function to be run in nanos. */ def executorDeserializeTime: Long = _executorDeserializeTime def executorDeserializeCpuTime: Long = _executorDeserializeCpuTime diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index e0276a4dc4224..54780b380e26d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -26,8 +26,9 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable import scala.collection.mutable.HashMap -import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.{Await, ExecutionContext, Future} import scala.concurrent.duration._ +import scala.concurrent.{Await, ExecutionContext, Future} import scala.reflect.ClassTag import scala.util.Random import scala.util.control.NonFatal @@ -125,7 +126,7 @@ private[spark] class BlockManager( val blockTransferService: BlockTransferService, securityManager: SecurityManager, numUsableCores: Int) - extends BlockDataManager with BlockEvictionHandler with Logging { + extends BlockDataManager with BlockEvictionHandler with Logging { private[spark] val externalShuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false) @@ -214,14 +215,14 @@ private[spark] class BlockManager( private val maxRemoteBlockToMem = conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM) /** - * Initializes the BlockManager with the given appId. This is not performed in the constructor as - * the appId may not be known at BlockManager instantiation time (in particular for the driver, - * where it is only learned after registration with the TaskScheduler). - * - * This method initializes the BlockTransferService and ShuffleClient, registers with the - * BlockManagerMaster, starts the BlockManagerWorker endpoint, and registers with a local shuffle - * service if configured. - */ + * Initializes the BlockManager with the given appId. This is not performed in the constructor as + * the appId may not be known at BlockManager instantiation time (in particular for the driver, + * where it is only learned after registration with the TaskScheduler). + * + * This method initializes the BlockTransferService and ShuffleClient, registers with the + * BlockManagerMaster, starts the BlockManagerWorker endpoint, and registers with a local shuffle + * service if configured. + */ def initialize(appId: String): Unit = { blockTransferService.init(this) shuffleClient.init(appId) @@ -290,25 +291,25 @@ private[spark] class BlockManager( } catch { case e: Exception if i < MAX_ATTEMPTS => logError(s"Failed to connect to external shuffle server, will retry ${MAX_ATTEMPTS - i}" - + s" more times after waiting $SLEEP_TIME_SECS seconds...", e) + + s" more times after waiting $SLEEP_TIME_SECS seconds...", e) Thread.sleep(SLEEP_TIME_SECS * 1000) case NonFatal(e) => throw new SparkException("Unable to register with external shuffle server due to : " + - e.getMessage, e) + e.getMessage, e) } } } /** - * Report all blocks to the BlockManager again. This may be necessary if we are dropped - * by the BlockManager and come back or if we become capable of recovering blocks on disk after - * an executor crash. - * - * This function deliberately fails silently if the master returns false (indicating that - * the slave needs to re-register). The error condition will be detected again by the next - * heart beat attempt or new block registration and another try to re-register all blocks - * will be made then. - */ + * Report all blocks to the BlockManager again. This may be necessary if we are dropped + * by the BlockManager and come back or if we become capable of recovering blocks on disk after + * an executor crash. + * + * This function deliberately fails silently if the master returns false (indicating that + * the slave needs to re-register). The error condition will be detected again by the next + * heart beat attempt or new block registration and another try to re-register all blocks + * will be made then. + */ private def reportAllBlocks(): Unit = { logInfo(s"Reporting ${blockInfoManager.size} blocks to the master.") for ((blockId, info) <- blockInfoManager.entries) { @@ -321,11 +322,11 @@ private[spark] class BlockManager( } /** - * Re-register with the master and report all blocks to it. This will be called by the heart beat - * thread if our heartbeat to the block manager indicates that we were not registered. - * - * Note that this method must be called without any BlockInfo locks held. - */ + * Re-register with the master and report all blocks to it. This will be called by the heart beat + * thread if our heartbeat to the block manager indicates that we were not registered. + * + * Note that this method must be called without any BlockInfo locks held. + */ def reregister(): Unit = { // TODO: We might need to rate limit re-registering. logInfo(s"BlockManager $blockManagerId re-registering with master") @@ -334,8 +335,8 @@ private[spark] class BlockManager( } /** - * Re-register with the master sometime soon. - */ + * Re-register with the master sometime soon. + */ private def asyncReregister(): Unit = { asyncReregisterLock.synchronized { if (asyncReregisterTask == null) { @@ -352,8 +353,8 @@ private[spark] class BlockManager( } /** - * For testing. Wait for any pending asynchronous re-registration; otherwise, do nothing. - */ + * For testing. Wait for any pending asynchronous re-registration; otherwise, do nothing. + */ def waitForAsyncReregister(): Unit = { val task = asyncReregisterTask if (task != null) { @@ -367,9 +368,9 @@ private[spark] class BlockManager( } /** - * Interface to get local block data. Throws an exception if the block cannot be found or - * cannot be read successfully. - */ + * Interface to get local block data. Throws an exception if the block cannot be found or + * cannot be read successfully. + */ override def getBlockData(blockId: BlockId): ManagedBuffer = { if (blockId.isShuffle) { shuffleManager.shuffleBlockResolver.getBlockData(blockId.asInstanceOf[ShuffleBlockId]) @@ -402,9 +403,9 @@ private[spark] class BlockManager( } /** - * Get the BlockStatus for the block identified by the given ID, if it exists. - * NOTE: This is mainly for testing. - */ + * Get the BlockStatus for the block identified by the given ID, if it exists. + * NOTE: This is mainly for testing. + */ def getStatus(blockId: BlockId): Option[BlockStatus] = { blockInfoManager.get(blockId).map { info => val memSize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L @@ -414,28 +415,28 @@ private[spark] class BlockManager( } /** - * Get the ids of existing blocks that match the given filter. Note that this will - * query the blocks stored in the disk block manager (that the block manager - * may not know of). - */ + * Get the ids of existing blocks that match the given filter. Note that this will + * query the blocks stored in the disk block manager (that the block manager + * may not know of). + */ def getMatchingBlockIds(filter: BlockId => Boolean): Seq[BlockId] = { // The `toArray` is necessary here in order to force the list to be materialized so that we // don't try to serialize a lazy iterator when responding to client requests. (blockInfoManager.entries.map(_._1) ++ diskBlockManager.getAllBlocks()) - .filter(filter) - .toArray - .toSeq + .filter(filter) + .toArray + .toSeq } /** - * Tell the master about the current storage status of a block. This will send a block update - * message reflecting the current status, *not* the desired storage level in its block info. - * For example, a block with MEMORY_AND_DISK set might have fallen out to be only on disk. - * - * droppedMemorySize exists to account for when the block is dropped from memory to disk (so - * it is still valid). This ensures that update in master will compensate for the increase in - * memory on slave. - */ + * Tell the master about the current storage status of a block. This will send a block update + * message reflecting the current status, *not* the desired storage level in its block info. + * For example, a block with MEMORY_AND_DISK set might have fallen out to be only on disk. + * + * droppedMemorySize exists to account for when the block is dropped from memory to disk (so + * it is still valid). This ensures that update in master will compensate for the increase in + * memory on slave. + */ private def reportBlockStatus( blockId: BlockId, status: BlockStatus, @@ -450,10 +451,10 @@ private[spark] class BlockManager( } /** - * Actually send a UpdateBlockInfo message. Returns the master's response, - * which will be true if the block was successfully recorded and false if - * the slave needs to re-register. - */ + * Actually send a UpdateBlockInfo message. Returns the master's response, + * which will be true if the block was successfully recorded and false if + * the slave needs to re-register. + */ private def tryToReportBlockStatus( blockId: BlockId, status: BlockStatus, @@ -465,10 +466,10 @@ private[spark] class BlockManager( } /** - * Return the updated storage status of the block with the given ID. More specifically, if - * the block is dropped from memory and possibly added to disk, return the new storage level - * and the updated in-memory and on-disk sizes. - */ + * Return the updated storage status of the block with the given ID. More specifically, if + * the block is dropped from memory and possibly added to disk, return the new storage level + * and the updated in-memory and on-disk sizes. + */ private def getCurrentBlockStatus(blockId: BlockId, info: BlockInfo): BlockStatus = { info.synchronized { info.level match { @@ -478,7 +479,7 @@ private[spark] class BlockManager( val inMem = level.useMemory && memoryStore.contains(blockId) val onDisk = level.useDisk && diskStore.contains(blockId) val deserialized = if (inMem) level.deserialized else false - val replication = if (inMem || onDisk) level.replication else 1 + val replication = if (inMem || onDisk) level.replication else 1 val storageLevel = StorageLevel( useDisk = onDisk, useMemory = inMem, @@ -493,19 +494,19 @@ private[spark] class BlockManager( } /** - * Get locations of an array of blocks. - */ + * Get locations of an array of blocks. + */ private def getLocationBlockIds(blockIds: Array[BlockId]): Array[Seq[BlockManagerId]] = { - val startTimeMs = System.currentTimeMillis + val startTimeMs = if (isDebugEnabled) System.currentTimeMillis else 0L val locations = master.getLocations(blockIds).toArray logDebug("Got multiple block location in %s".format(Utils.getUsedTimeMs(startTimeMs))) locations } /** - * Cleanup code run in response to a failed local read. - * Must be called while holding a read lock on the block. - */ + * Cleanup code run in response to a failed local read. + * Must be called while holding a read lock on the block. + */ private def handleLocalReadFailure(blockId: BlockId): Nothing = { releaseLock(blockId) // Remove the missing block so that its unavailability is reported to the driver @@ -514,8 +515,8 @@ private[spark] class BlockManager( } /** - * Get block from local block manager as an iterator of Java objects. - */ + * Get block from local block manager as an iterator of Java objects. + */ def getLocalValues(blockId: BlockId): Option[BlockResult] = { logDebug(s"Getting local block $blockId") blockInfoManager.lockForReading(blockId) match { @@ -610,7 +611,8 @@ private[spark] class BlockManager( } else { handleLocalReadFailure(blockId) } - } else { // storage level is serialized + } else { + // storage level is serialized if (level.useMemory && memoryStore.contains(blockId)) { new ByteBufferBlockData(memoryStore.getBytes(blockId).get, false) } else if (level.useDisk && diskStore.contains(blockId)) { @@ -625,10 +627,10 @@ private[spark] class BlockManager( } /** - * Get block from remote block managers. - * - * This does not acquire a lock on this block in this JVM. - */ + * Get block from remote block managers. + * + * This does not acquire a lock on this block in this JVM. + */ private def getRemoteValues[T: ClassTag](blockId: BlockId): Option[BlockResult] = { val ct = implicitly[ClassTag[T]] getRemoteBytes(blockId).map { data => @@ -656,8 +658,8 @@ private[spark] class BlockManager( } /** - * Get block from remote block managers as serialized bytes. - */ + * Get block from remote block managers as serialized bytes. + */ def getRemoteBytes(blockId: BlockId): Option[ChunkedByteBuffer] = { logDebug(s"Getting remote block $blockId") require(blockId != null, "BlockId is null") @@ -700,12 +702,12 @@ private[spark] class BlockManager( // or we've refreshed the list of locations from the master, and have still // hit failures after trying locations from the refreshed list. logWarning(s"Failed to fetch block after $totalFailureCount fetch failures. " + - s"Most recent failure cause:", e) + s"Most recent failure cause:", e) return None } logWarning(s"Failed to fetch remote block $blockId " + - s"from $loc (failed attempt $runningFailureCount)", e) + s"from $loc (failed attempt $runningFailureCount)", e) // If there is a large number of executors then locations list can contain a // large number of stale entries causing a large number of retries that may @@ -714,7 +716,7 @@ private[spark] class BlockManager( if (runningFailureCount >= maxFailuresBeforeLocationRefresh) { locationIterator = sortLocations(master.getLocations(blockId)).iterator logDebug(s"Refreshed locations from the driver " + - s"after ${runningFailureCount} fetch failures.") + s"after ${runningFailureCount} fetch failures.") runningFailureCount = 0 } @@ -732,12 +734,12 @@ private[spark] class BlockManager( } /** - * Get a block from the block manager (either local or remote). - * - * This acquires a read lock on the block if the block was stored locally and does not acquire - * any locks if the block was fetched from a remote block manager. The read lock will - * automatically be freed once the result's `data` iterator is fully consumed. - */ + * Get a block from the block manager (either local or remote). + * + * This acquires a read lock on the block if the block was stored locally and does not acquire + * any locks if the block was fetched from a remote block manager. The read lock will + * automatically be freed once the result's `data` iterator is fully consumed. + */ def get[T: ClassTag](blockId: BlockId): Option[BlockResult] = { val local = getLocalValues(blockId) if (local.isDefined) { @@ -753,8 +755,8 @@ private[spark] class BlockManager( } /** - * Downgrades an exclusive write lock to a shared read lock. - */ + * Downgrades an exclusive write lock to a shared read lock. + */ def downgradeLock(blockId: BlockId): Unit = { blockInfoManager.downgradeLock(blockId) } @@ -770,28 +772,28 @@ private[spark] class BlockManager( } /** - * Registers a task with the BlockManager in order to initialize per-task bookkeeping structures. - */ + * Registers a task with the BlockManager in order to initialize per-task bookkeeping structures. + */ def registerTask(taskAttemptId: Long): Unit = { blockInfoManager.registerTask(taskAttemptId) } /** - * Release all locks for the given task. - * - * @return the blocks whose locks were released. - */ + * Release all locks for the given task. + * + * @return the blocks whose locks were released. + */ def releaseAllLocksForTask(taskAttemptId: Long): Seq[BlockId] = { blockInfoManager.releaseAllLocksForTask(taskAttemptId) } /** - * Retrieve the given block if it exists, otherwise call the provided `makeIterator` method - * to compute the block, persist it, and return its values. - * - * @return either a BlockResult if the block was successfully cached, or an iterator if the block - * could not be cached. - */ + * Retrieve the given block if it exists, otherwise call the provided `makeIterator` method + * to compute the block, persist it, and return its values. + * + * @return either a BlockResult if the block was successfully cached, or an iterator if the block + * could not be cached. + */ def getOrElseUpdate[T]( blockId: BlockId, level: StorageLevel, @@ -803,7 +805,7 @@ private[spark] class BlockManager( case Some(block) => return Left(block) case _ => - // Need to compute the block. + // Need to compute the block. } // Initially we hold no locks on this block. doPutIterator(blockId, makeIterator, level, classTag, keepReadLock = true) match { @@ -825,13 +827,13 @@ private[spark] class BlockManager( // The put failed, likely because the data was too large to fit in memory and could not be // dropped to disk. Therefore, we need to pass the input iterator back to the caller so // that they can decide what to do with the values (e.g. process them without caching). - Right(iter) + Right(iter) } } /** - * @return true if the block was stored or false if an error occurred. - */ + * @return true if the block was stored or false if an error occurred. + */ def putIterator[T: ClassTag]( blockId: BlockId, values: Iterator[T], @@ -850,10 +852,10 @@ private[spark] class BlockManager( } /** - * A short circuited method to get a block writer that can write data directly to disk. - * The Block will be appended to the File specified by filename. Callers should handle error - * cases. - */ + * A short circuited method to get a block writer that can write data directly to disk. + * The Block will be appended to the File specified by filename. Callers should handle error + * cases. + */ def getDiskWriter( blockId: BlockId, file: File, @@ -904,7 +906,7 @@ private[spark] class BlockManager( tellMaster: Boolean = true, keepReadLock: Boolean = false): Boolean = { doPut(blockId, level, classTag, tellMaster = tellMaster, keepReadLock = keepReadLock) { info => - val startTimeMs = System.currentTimeMillis + val startTimeMs = if (isDebugEnabled) System.currentTimeMillis else 0L // Since we're storing bytes, initiate the replication before storing them locally. // This is faster as data is already serialized and ready to send. val replicationFuture = if (level.replication > 1) { @@ -983,11 +985,11 @@ private[spark] class BlockManager( } /** - * Helper method used to abstract common code from [[doPutBytes()]] and [[doPutIterator()]]. - * - * @param putBody a function which attempts the actual put() and returns None on success - * or Some on failure. - */ + * Helper method used to abstract common code from [[doPutBytes()]] and [[doPutIterator()]]. + * + * @param putBody a function which attempts the actual put() and returns None on success + * or Some on failure. + */ private def doPut[T]( blockId: BlockId, level: StorageLevel, @@ -1012,7 +1014,7 @@ private[spark] class BlockManager( } } - val startTimeMs = System.currentTimeMillis + val startTimeMs = if (isDebugEnabled) System.currentTimeMillis else 0L var exceptionWasThrown: Boolean = true val result: Option[T] = try { val res = putBody(putBlockInfo) @@ -1053,26 +1055,26 @@ private[spark] class BlockManager( } if (level.replication > 1) { logDebug("Putting block %s with replication took %s" - .format(blockId, Utils.getUsedTimeMs(startTimeMs))) + .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } else { logDebug("Putting block %s without replication took %s" - .format(blockId, Utils.getUsedTimeMs(startTimeMs))) + .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } result } /** - * Put the given block according to the given level in one of the block stores, replicating - * the values if necessary. - * - * If the block already exists, this method will not overwrite it. - * - * @param keepReadLock if true, this method will hold the read lock when it returns (even if the - * block already exists). If false, this method will hold no locks when it - * returns. - * @return None if the block was already present or if the put succeeded, or Some(iterator) - * if the put failed. - */ + * Put the given block according to the given level in one of the block stores, replicating + * the values if necessary. + * + * If the block already exists, this method will not overwrite it. + * + * @param keepReadLock if true, this method will hold the read lock when it returns (even if the + * block already exists). If false, this method will hold no locks when it + * returns. + * @return None if the block was already present or if the put succeeded, or Some(iterator) + * if the put failed. + */ private def doPutIterator[T]( blockId: BlockId, iterator: () => Iterator[T], @@ -1081,7 +1083,7 @@ private[spark] class BlockManager( tellMaster: Boolean = true, keepReadLock: Boolean = false): Option[PartiallyUnrolledIterator[T]] = { doPut(blockId, level, classTag, tellMaster = tellMaster, keepReadLock = keepReadLock) { info => - val startTimeMs = System.currentTimeMillis + val startTimeMs = if (isDebugEnabled) System.currentTimeMillis else 0L var iteratorFromFailedMemoryStorePut: Option[PartiallyUnrolledIterator[T]] = None // Size of the block in bytes var size = 0L @@ -1105,7 +1107,8 @@ private[spark] class BlockManager( iteratorFromFailedMemoryStorePut = Some(iter) } } - } else { // !level.deserialized + } else { + // !level.deserialized memoryStore.putIteratorAsBytes(blockId, iterator(), classTag, level.memoryMode) match { case Right(s) => size = s @@ -1143,7 +1146,7 @@ private[spark] class BlockManager( addUpdatedBlockStatusToTaskMetrics(blockId, putBlockStatus) logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) if (level.replication > 1) { - val remoteStartTime = System.currentTimeMillis + val remoteStartTime = if (isDebugEnabled) System.currentTimeMillis else 0L val bytesToReplicate = doGetLocalBytes(blockId, info) // [SPARK-16550] Erase the typed classTag when using default serialization, since // NettyBlockRpcServer crashes when deserializing repl-defined classes. @@ -1159,7 +1162,7 @@ private[spark] class BlockManager( bytesToReplicate.dispose() } logDebug("Put block %s remotely took %s" - .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) + .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) } } assert(blockWasSuccessfullyStored == iteratorFromFailedMemoryStorePut.isEmpty) @@ -1168,14 +1171,14 @@ private[spark] class BlockManager( } /** - * Attempts to cache spilled bytes read from disk into the MemoryStore in order to speed up - * subsequent reads. This method requires the caller to hold a read lock on the block. - * - * @return a copy of the bytes from the memory store if the put succeeded, otherwise None. - * If this returns bytes from the memory store then the original disk store bytes will - * automatically be disposed and the caller should not continue to use them. Otherwise, - * if this returns None then the original disk store bytes will be unaffected. - */ + * Attempts to cache spilled bytes read from disk into the MemoryStore in order to speed up + * subsequent reads. This method requires the caller to hold a read lock on the block. + * + * @return a copy of the bytes from the memory store if the put succeeded, otherwise None. + * If this returns bytes from the memory store then the original disk store bytes will + * automatically be disposed and the caller should not continue to use them. Otherwise, + * if this returns None then the original disk store bytes will be unaffected. + */ private def maybeCacheDiskBytesInMemory( blockInfo: BlockInfo, blockId: BlockId, @@ -1215,12 +1218,12 @@ private[spark] class BlockManager( } /** - * Attempts to cache spilled values read from disk into the MemoryStore in order to speed up - * subsequent reads. This method requires the caller to hold a read lock on the block. - * - * @return a copy of the iterator. The original iterator passed this method should no longer - * be used after this method returns. - */ + * Attempts to cache spilled values read from disk into the MemoryStore in order to speed up + * subsequent reads. This method requires the caller to hold a read lock on the block. + * + * @return a copy of the iterator. The original iterator passed this method should no longer + * be used after this method returns. + */ private def maybeCacheDiskValuesInMemory[T]( blockInfo: BlockInfo, blockId: BlockId, @@ -1252,8 +1255,8 @@ private[spark] class BlockManager( } /** - * Get peer block managers in the system. - */ + * Get peer block managers in the system. + */ private def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = { peerFetchLock.synchronized { val cachedPeersTtl = conf.getInt("spark.storage.cachedPeersTtl", 60 * 1000) // milliseconds @@ -1319,7 +1322,7 @@ private[spark] class BlockManager( replication = 1) val numPeersToReplicateTo = level.replication - 1 - val startTime = System.nanoTime + val startTime = if (isDebugEnabled) System.nanoTime else 0L val peersReplicatedTo = mutable.HashSet.empty ++ existingReplicas val peersFailedToReplicateTo = mutable.HashSet.empty[BlockManagerId] @@ -1337,9 +1340,10 @@ private[spark] class BlockManager( while(numFailures <= maxReplicationFailures && !peersForReplication.isEmpty && peersReplicatedTo.size < numPeersToReplicateTo) { + val peer = peersForReplication.head try { - val onePeerStartTime = System.nanoTime + val onePeerStartTime = if (isDebugEnabled) System.nanoTime else 0L logTrace(s"Trying to replicate $blockId of ${data.size} bytes to $peer") blockTransferService.uploadBlockSync( peer.host, @@ -1350,7 +1354,7 @@ private[spark] class BlockManager( tLevel, classTag) logTrace(s"Replicated $blockId of ${data.size} bytes to $peer" + - s" in ${(System.nanoTime - onePeerStartTime).toDouble / 1e6} ms") + s" in ${(System.nanoTime - onePeerStartTime).toDouble / 1e6} ms") peersForReplication = peersForReplication.tail peersReplicatedTo += peer } catch { @@ -1374,28 +1378,28 @@ private[spark] class BlockManager( } } logDebug(s"Replicating $blockId of ${data.size} bytes to " + - s"${peersReplicatedTo.size} peer(s) took ${(System.nanoTime - startTime) / 1e6} ms") + s"${peersReplicatedTo.size} peer(s) took ${(System.nanoTime - startTime) / 1e6} ms") if (peersReplicatedTo.size < numPeersToReplicateTo) { logWarning(s"Block $blockId replicated to only " + - s"${peersReplicatedTo.size} peer(s) instead of $numPeersToReplicateTo peers") + s"${peersReplicatedTo.size} peer(s) instead of $numPeersToReplicateTo peers") } logDebug(s"block $blockId replicated to ${peersReplicatedTo.mkString(", ")}") } /** - * Read a block consisting of a single object. - */ + * Read a block consisting of a single object. + */ def getSingle[T: ClassTag](blockId: BlockId): Option[T] = { get[T](blockId).map(_.data.next().asInstanceOf[T]) } /** - * Write a block consisting of a single object. - * - * @return true if the block was stored or false if the block was already stored or an - * error occurred. - */ + * Write a block consisting of a single object. + * + * @return true if the block was stored or false if the block was already stored or an + * error occurred. + */ def putSingle[T: ClassTag]( blockId: BlockId, value: T, @@ -1405,16 +1409,16 @@ private[spark] class BlockManager( } /** - * Drop a block from memory, possibly putting it on disk if applicable. Called when the memory - * store reaches its limit and needs to free up space. - * - * If `data` is not put on disk, it won't be created. - * - * The caller of this method must hold a write lock on the block before calling this method. - * This method does not release the write lock. - * - * @return the block's new effective StorageLevel. - */ + * Drop a block from memory, possibly putting it on disk if applicable. Called when the memory + * store reaches its limit and needs to free up space. + * + * If `data` is not put on disk, it won't be created. + * + * The caller of this method must hold a write lock on the block before calling this method. + * This method does not release the write lock. + * + * @return the block's new effective StorageLevel. + */ private[storage] override def dropFromMemory[T: ClassTag]( blockId: BlockId, data: () => Either[Array[T], ChunkedByteBuffer]): StorageLevel = { @@ -1462,10 +1466,10 @@ private[spark] class BlockManager( } /** - * Remove all blocks belonging to the given RDD. - * - * @return The number of blocks removed. - */ + * Remove all blocks belonging to the given RDD. + * + * @return The number of blocks removed. + */ def removeRdd(rddId: Int): Int = { // TODO: Avoid a linear scan by creating another mapping of RDD.id to blocks. logInfo(s"Removing RDD $rddId") @@ -1475,20 +1479,20 @@ private[spark] class BlockManager( } /** - * Remove all blocks belonging to the given broadcast. - */ + * Remove all blocks belonging to the given broadcast. + */ def removeBroadcast(broadcastId: Long, tellMaster: Boolean): Int = { logDebug(s"Removing broadcast $broadcastId") val blocksToRemove = blockInfoManager.entries.map(_._1).collect { - case bid @ BroadcastBlockId(`broadcastId`, _) => bid + case bid@BroadcastBlockId(`broadcastId`, _) => bid } blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster) } blocksToRemove.size } /** - * Remove a block from both memory and disk. - */ + * Remove a block from both memory and disk. + */ def removeBlock(blockId: BlockId, tellMaster: Boolean = true): Unit = { logDebug(s"Removing block $blockId") blockInfoManager.lockForWriting(blockId) match { @@ -1502,9 +1506,9 @@ private[spark] class BlockManager( } /** - * Internal version of [[removeBlock()]] which assumes that the caller already holds a write - * lock on the block. - */ + * Internal version of [[removeBlock()]] which assumes that the caller already holds a write + * lock on the block. + */ private def removeBlockInternal(blockId: BlockId, tellMaster: Boolean): Unit = { // Removals are idempotent in disk store and memory store. At worst, we get a warning. val removedFromMemory = memoryStore.remove(blockId) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 39249d411b582..e4c11555bbe09 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -60,7 +60,7 @@ private[spark] class DiskStore( throw new IllegalStateException(s"Block $blockId is already present in the disk store") } logDebug(s"Attempting to put block $blockId") - val startTime = System.currentTimeMillis + val startTime = if (isDebugEnabled) System.currentTimeMillis else 0L val file = diskManager.getFile(blockId) val out = new CountingWritableChannel(openForWrite(file)) var threwException: Boolean = true @@ -83,11 +83,10 @@ private[spark] class DiskStore( } } } - val finishTime = System.currentTimeMillis logDebug("Block %s stored as %s file on disk in %d ms".format( file.getName, Utils.bytesToString(file.length()), - finishTime - startTime)) + System.currentTimeMillis - startTime)) } def putBytes(blockId: BlockId, bytes: ChunkedByteBuffer): Unit = { diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 98b5a735a4529..5df11ddcaad06 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -87,7 +87,8 @@ final class ShuffleBlockFetcherIterator( */ private[this] var numBlocksProcessed = 0 - private[this] val startTime = System.currentTimeMillis + private[this] val startTime = + if (isDebugEnabled || isTraceEnabled) System.currentTimeMillis else 0L /** Local blocks to fetch, excluding zero-sized blocks. */ private[this] val localBlocks = new ArrayBuffer[BlockId]() @@ -231,8 +232,12 @@ final class ShuffleBlockFetcherIterator( remainingBlocks -= blockId results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf, remainingBlocks.isEmpty)) - logDebug("remainingBlocks: " + remainingBlocks) + if (isDebugEnabled) { + logDebug("remainingBlocks: " + remainingBlocks) + } } + if (isTraceEnabled) logTrace("Got remote block " + blockId + " after " + + Utils.getUsedTimeMs(startTime)) } logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) } @@ -351,11 +356,12 @@ final class ShuffleBlockFetcherIterator( fetchUpToMaxBytes() val numFetches = remoteRequests.size - fetchRequests.size - logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime)) + if (isDebugEnabled) logDebug("Started " + numFetches + " remote fetches in" + + Utils.getUsedTimeMs(startTime)) // Get Local Blocks fetchLocalBlocks() - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime)) + if (isDebugEnabled) logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime)) } override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch @@ -382,10 +388,10 @@ final class ShuffleBlockFetcherIterator( // is also corrupt, so the previous stage could be retried. // For local shuffle block, throw FailureFetchResult for the first IOException. while (result == null) { - val startFetchWait = System.currentTimeMillis() + val startFetchWait = System.nanoTime() result = results.take() - val stopFetchWait = System.currentTimeMillis() - shuffleMetrics.incFetchWaitTime(stopFetchWait - startFetchWait) + val stopFetchWait = System.nanoTime() + shuffleMetrics.incFetchWaitTime(math.max(stopFetchWait - startFetchWait, 0L) / 1000000.0) result match { case r @ SuccessFetchResult(blockId, address, size, buf, isNetworkReqDone) => diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index ff83301d631c4..ea811f45086c2 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -324,8 +324,9 @@ private[spark] object JsonProtocol { value match { case v: Int => JInt(v) case v: Long => JInt(v) - // We only have 3 kind of internal accumulator types, so if it's not int or long, it must be - // the blocks accumulator, whose type is `java.util.List[(BlockId, BlockStatus)]` + case v: Double => JDouble(v) + // We only have 4 kinds of internal accumulator types, so if it's not int, long or double, + // it must be the blocks accumulator with type `java.util.List[(BlockId, BlockStatus)]` case v => JArray(v.asInstanceOf[java.util.List[(BlockId, BlockStatus)]].asScala.toList.map { case (id, status) => From e25e4b91f8b3e3c044f9ba3e6e28760f31de1947 Mon Sep 17 00:00:00 2001 From: Rishitesh Mishra Date: Thu, 2 Feb 2017 07:16:00 +0530 Subject: [PATCH 051/142] Snap 982 (#43) * a) Added a method in SparkContext to manipulate addedJar. This is an workaround for SNAP-1133. b) made repl classloader a variable in Executor.scala * Changed Executor field variable to protected. * Changed build.gradle of launcher and network-yarn to exclude netty dependecies , which was causing some messages to hang. made urlclassLoader in Executor.scala a variable. * Made Utils.doFetchFile method public. * Made Executor.addReplClassLoaderIfNeeded() method as public. --- common/network-yarn/build.gradle | 2 ++ core/src/main/scala/org/apache/spark/SparkContext.scala | 5 +++++ .../src/main/scala/org/apache/spark/executor/Executor.scala | 6 +++--- launcher/build.gradle | 2 ++ 4 files changed, 12 insertions(+), 3 deletions(-) diff --git a/common/network-yarn/build.gradle b/common/network-yarn/build.gradle index bbb6d8c7f81ae..b447d5aecaeaa 100644 --- a/common/network-yarn/build.gradle +++ b/common/network-yarn/build.gradle @@ -41,6 +41,8 @@ dependencies { exclude(group: 'com.sun.jersey') exclude(group: 'com.sun.jersey.jersey-test-framework') exclude(group: 'com.sun.jersey.contribs') + exclude(group: 'io.netty', module: 'netty') + exclude(group: 'io.netty', module: 'netty-all') } /* diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index fcc5fce4bb798..3a0c093587157 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -280,6 +280,11 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] val addedFiles = new ConcurrentHashMap[String, Long]().asScala private[spark] val addedJars = new ConcurrentHashMap[String, Long]().asScala + def removeAddedJar(name : String) { + logInfo(s"Removing jar $name from SparkContext list") + addedJars.remove(name) + } + // Keeps track of all persisted RDDs private[spark] val persistentRdds = { val map: ConcurrentMap[Int, RDD[_]] = new MapMaker().weakValues().makeMap[Int, RDD[_]]() diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 80da4e1a6f1dc..f8cf7a0943584 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -126,8 +126,8 @@ private[spark] class Executor( // Create our ClassLoader // do this after SparkEnv creation so can access the SecurityManager - protected val urlClassLoader = createClassLoader() - private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader) + protected var urlClassLoader = createClassLoader() + protected var replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader) // Set the classloader for serializer env.serializer.setDefaultClassLoader(replClassLoader) @@ -726,7 +726,7 @@ private[spark] class Executor( * If the REPL is in use, add another ClassLoader that will read * new classes defined by the REPL as the user types code */ - private def addReplClassLoaderIfNeeded(parent: ClassLoader): ClassLoader = { + def addReplClassLoaderIfNeeded(parent: ClassLoader): ClassLoader = { val classUri = conf.get("spark.repl.class.uri", null) if (classUri != null) { logInfo("Using REPL class URI: " + classUri) diff --git a/launcher/build.gradle b/launcher/build.gradle index 22a32f5227a2a..185fb1bdd875d 100644 --- a/launcher/build.gradle +++ b/launcher/build.gradle @@ -34,6 +34,8 @@ dependencies { exclude(group: 'com.sun.jersey') exclude(group: 'com.sun.jersey.jersey-test-framework') exclude(group: 'com.sun.jersey.contribs') + exclude(group: 'io.netty', module: 'netty') + exclude(group: 'io.netty', module: 'netty-all') } testCompile group: 'org.slf4j', name: 'jul-to-slf4j', version: slf4jVersion } From 1694567e7386d4de0de979f820f5b6c9bef3487d Mon Sep 17 00:00:00 2001 From: snappy-sachin Date: Tue, 8 Aug 2017 16:59:05 +0530 Subject: [PATCH 052/142] Snap 1890 : Snappy Pulse UI suggestions for 1.0 (#69) * SNAP-1890: Snappy Pulse UI suggestions for 1.0 Changes: - SnappyData logo shifted to right most side on navigation tab bar. - Adding SnappyData's own new Pulse logo on left most side on navigation tab bar. - Displaying SnappyData Build details along with product version number on Pulse UI. - Adding CSS,HTML, JS code changes for displaying version details pop up. Conflicts: core/src/main/scala/org/apache/spark/ui/UIUtils.scala --- .../spark/ui/static/snappy-dashboard.css | 2 +- .../snappydata/pulse-snappydata-130X50.png | Bin 0 -> 4650 bytes .../snappydata/pulse-snappydata-152X50.png | Bin 0 -> 9070 bytes .../static/snappydata/snappydata-310X50.png | Bin 0 -> 7750 bytes .../org/apache/spark/ui/static/webui.css | 78 +++++++++++++++++- .../org/apache/spark/ui/static/webui.js | 10 ++- .../scala/org/apache/spark/ui/SparkUI.scala | 9 +- .../scala/org/apache/spark/ui/UIUtils.scala | 63 +++++++++++--- 8 files changed, 142 insertions(+), 20 deletions(-) create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/pulse-snappydata-130X50.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/pulse-snappydata-152X50.png create mode 100644 core/src/main/resources/org/apache/spark/ui/static/snappydata/snappydata-310X50.png diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappy-dashboard.css b/core/src/main/resources/org/apache/spark/ui/static/snappy-dashboard.css index 1ddd1b5263e45..73e740086cb16 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/snappy-dashboard.css +++ b/core/src/main/resources/org/apache/spark/ui/static/snappy-dashboard.css @@ -114,6 +114,6 @@ }*/ .progressValue{ float:right; - width:20%; + /* width:20%; */ text-align:center; } \ No newline at end of file diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/pulse-snappydata-130X50.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/pulse-snappydata-130X50.png new file mode 100644 index 0000000000000000000000000000000000000000..8de1e13897efa4451cd2ba0de0135368f4b325c9 GIT binary patch literal 4650 zcmV+_64mXAP)UW2Nt*xw02y>e zSad^gZEa<4bO1wgWnpw>WFU8GbZ8()Nlj2!fese{01=W&L_t(|+U=ZaU{po6$A8tG z1(J{r2w^8|Q5bdwK|n-BKvWFrx}pQ9<0$9|sJH=&qo9rgOvbYVpEvNQRUb{OzxNRokp=oidiU&Az3^&1QLK&z>mNhQ>jCh}0g??V zgDWvhhZ|Dv1daf20#90fIA};#?zMgdc-!72d80KXVg_NZW>-POP(-oXy6Xa*ZnP5_ny zZGq*$5X;*b1kA$|ka`FxG^D(?ycums*$2o13?R<}payUgFctV7Pta<1FH=w z#{u_SJU$J4RmP&+418>5Ng&x&s??{Q6~GKb%4lFKFbr62DmAbsEfxOU$HA5o`~b8v zmCE(g;Zjqnxxg*f^S=!#gD>K28Dm+m2TY}M{d~8?;<1sb)W9mftRdKZcR0vHK6}T+JEkMeSi1gLxlZa1#(}1)IiMuHzs)XBf00<-5Rx zmUrQ;6Kgfo{a3PaswP{6Ce?K3yn&^{i{(Q4q=go)dCriszGanq0?z{X0;2(GD&;-e zR$!(5f$ysM{nIslX=qtZA9<*i9xRiuD}hPMQc#(?RS{6GA!V+q)CS-+L&_I{CxCw$ zQbw6dxh&ThpkYlC=zjw&-c<>}RUVoKSYM{dAuEll0?gj)D(tc9h}PXYh5mXDEE zk!_Ze-BX<*A>}nyRQL~pnU{(p#2Hd9smMR~QG%|(t9Z@`+)>Wf^GfUSHABk4O3dV! zt;gvX5X=pRlnpAfTs})&7b|P4&+BWgRB?fmP(FQq!m=s@YLX&|GM#M>900xm>f_mp zU2Q5gsDc6N1FW;!y4O@{zMpdMSfDKx=xi!gTuzw)JR3;EES%BORO$>y@ZE;aPU>O@ z_E@iO0Y34%yk+r?6+B9_yoN|TVQ25VzXKETL{M7*hy7VCL(1{MKY#*Lsqjld5!U0m zUY2U1?_-v7ya#LoVu0(d^|g~#-C-(~R36CmnN6SK*~iEO-U41XmC7)rj0FCSXP3~+ zLwi%HBHwn(o<}gG^xZJpYw$Xm}M%p#wybc zxZUEdg{4gGOr;Lgq)jKkieM_$$I6>w38|rX=IP2ewxhw3$jhYAPkHnYszjy@hQS#)vePx~)7VuxJ&Ykn$0q_nD?r zBds>?!E;U`4>%4i1Nxdu^|F?)^JcJ5EaH7rspeLIDcPD3$1UEb0(MiWkv>tynoEY1 z%?&BFA?1Jkpyl5r8d#n87vr*PO+S}deMJ~DgsJzh`2WS^@D%i8=Vvc)w5U||i3u$# z{hf2j;Yr#$rkq_)@2=kc+o3XH++Rnw5zz@BSF3(oc}w&}N7vgfMr6@LpMJ{pq5Xoc zmw#T>faBNLbWz3?`rDfyNy^^xf2Ig2!x6wR>CmBwQbCEkkyzfBR5^|+>#(!#vqQHG7kM@++H4yBmfu$&jz+9d z$(^2EmsJW7E)JeKI(Ow?%4=)dd0A=d5n)xd2!Kr& zN4v%8V+)roECdR_0OAv$=rGi_ZT3DL85At27eYbI?7X3US0)u5>>wX`rHf7Ib9ESB$Tc^m0XwQ6ob|PYf2X87yg*_mxf(OxMzm8sQ;!z z_P>(-FS|uRmR-BrwsNnPFZ}!Hx~Af;;XAqNxRmv}N4fXBvR(9=-h!N17j;9TIlTWJ z=T}3t264{N+C?}&W&R+dAF@0XE*BGKMd+fXB|;|vbZ|V-yDdjmh!6+dxSG_}Y0v!A zdH=#b^3KKmbzaFn!kfDYY}QC*>tQn zGb0}#=CFTNPmobUX~R&x_q7Gizs~6;$2>ifLxF3EaWv3--uX%oT|P`KHD@|^bh|up zQs*JIA79$1BkKeT!b2%&FgtJ9)g%=ij&(dcsK3}TOB1}O6jwro{^i^E3Rev%UUH(S z%8Ia&hwp91^V=UzY#7>99R1rQ=k8+yTj^n4eB(#XMFQY*`QWEPqT?vo^1@Cv`>~y# zGYPbDO#0}$fUMhK^J^eZXS(-&zAEYaw*an@rF6QTISC3^@TMW@*BkK*x z&QdEwv*q16Uyj()d*G6j>%-+|!`kuUs+G$*ClFP*G#d~)tIOTl$EXvP=A4snRMJ`>m{nq#qu6G>=md?laOF9KTmGjf?09VOT zId>QCipFy%1Q!8pb-u8mxg7fL$7<yACQjtk8Uj(fN{2lA2~`@24+IUP&(25+ST9h2%3fnflRM!epw zr~TX6b#Zml91bhefnR!!%K%%}_Em!}gH^lvS%1D6okcp4 zpcr@G7w6i4dJNVjbmP5Qm+4(k`d{PR_7XG!w|y8hDB!cXA>wpD8~ekG2~6`!RU^(U zggITKsjd8FQsCCxb&oeDh;8Fqxn6p)dr-#WmA1MiXBX>(^@Xy| zVf-LUe{O%@zWqcubYly=jP%vsaz(mc`iTHt;lzt3{SRIx0NAf%Hvd5LHg4?p)}*DB zy9|J6OxF6dvS^rEY{?==xA;8U*6qoJ#JI3xap1NXoj1Qy7{R&~be$D}Bh&du0NOVn zcyhg(Ib$0!YPsnACpR{1=<~L2QxV}EH=dg2dE{k^b>}y_cwr+!T{T`j_uh!bhKG`Okg1y2so@M)`gE<;mPv@l9*KJnsvPGU~&Vl}Jg!g#3yF|2>ckXN* zeuI3tep`{B(Ak|fA0U_5CM%3~-3@t1y*OzOYDI3INb&@3j?sGIylm9%Qiipc?_b`Ytjn(>ZTE%R@ONONm3_~@ z8tJU1Hf+l?bhGTY9i2j*PUz92s$N4KA@YcrkR_sWBJ7!YSBo7hW~$x)5NI3VdlfAR z2;QAHXWMSNZ~e)3>{JJxwJeMK?5?5eM?m|p_XIuO1)J@1k2TzP}brVIM%!jv6c+1Dl9`9t*S2TzFhfA=G} z0B=7FDP7T$-Dy3SP>AO)MF_y!qPMSqOJ6_Mg~%Vk{P&-6t!*(vxr(eY3GyAy^1hMf z?lo#xK*xZ8(WTax^nPr?+2BUTb*)#W0eiW@oh*r zrBMZ<;Xtjmagvxx>xujcVDoSB#{o}|zUHiJ=ZF(=jwxp~Q~N(mG(Z|fvm6mumJ_&% zxX*FrPJi{9JEmNFkG%TOg#t>-5liUJzdA%wZb0eo5k1fRo;;=(7YZn~A3oQ2jEM6I zLd_d>wp`@!^mFRDleF}wThvOo(z6Z@CBK?|NVkm55n-}4ElmGw#07*qoM6N<$f`f_a$p8QV literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/pulse-snappydata-152X50.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/pulse-snappydata-152X50.png new file mode 100644 index 0000000000000000000000000000000000000000..431452c2ea40be39eb8eb54a0f807e0477503688 GIT binary patch literal 9070 zcmV-!Baz&RP)YM-o02y>e zSad^gZEa<4bO1wgWnpw>WFU8GbZ8()Nlj2!fese{03ZNKL_t(|+U=cpm=smk|39}v zr^ymUW&i^Uf)N!=prF8zM3}CT90W72VSROX4XdlWueJ zpnyn}%)rEsRds)VRL{^8Kn31if3KY9>8Go2Ro!sz_nvd^Ij5@OI2{!QigRkdlaG&+ zOX4^Sjv2@lKmvqvx5_|q4x!wwzj@wh$Ki2INKJeMNGDL-Djv$+a-1v20d};KD;pdI zaC{&-0L?&r2(r9dn*G*EInIKkWS=GlQUrirz#t^)F=EHTbzFiS3zFa<5CkNgC%}Wi zgr+K#JlDsJjSb9Ul#Y&i8u36^=W~*aLxE&CtfUC+2Hm4jgg(a zM9?>=O%vJ*Q)T%mLN_w*c!xxm%rvc9LC<34CZi4L~1bbZZ~6^A((v z1Tr9yF$HIvkMO52nG3fvT=HPFBY!ShU5_Vc<^6$gW<3T#+u(zG2eIKo1?O}H*$5;8 zxmzdx^$c_F*8n4pUKE&b9tT)5wK47%?b&@@0PAUiOch8{DIYo0?l!lVy6Pnr>nZ`M zfHxsY;y_5j-jFU!M!s0|{+jEJ-b^tU*Q)ni;{ktbHWuhueE^?=9Z8cvw4Ge^#rEXu zn&th2-Lvkdk1=LcfbcT{=@&>#Cv-!W^L4@0(XOoz^pWt4fUkf%fc1b~7hFAzn|arK z4vZDrKqkJdVf9Mte>FuxhCD%FT{|G?4;JvgFI|ii?n=NWADM2QCZ=YSc`GW(b12+Rhk!(WnU%>Z3JQ-kT1Rr5-(fxEcZaE{91D~iknLjl(KB``t zZY~&8vuhpLm@&TbOq}@`J&kegQwS~4uuu7EJIINoQg-DO^Q}+J!A&t{?>TORxl36+ zODA?h9mb#WQ{buwO(khl#~N3+eWb~@gj*qp?~4A0iX15FtTI7-dqee{E`AuTCxI(r z+V2y@BoZOOBcUU(5rI<8(`s;4d-c~A-Om-qjVESs;EkUKm)lAdfA4#Or#G%H>E+F5 z;>^d$4GzmsqiXrj0T+OS;;SU><=cYYGmX?a$Bq1IN~ff^I0_-_pf---yPipcYI~^h z=PiVays);cu*i}--cK&m^R5QKN3^G`3orQU_OfLT0wED0eH~ zYX&EDY4lzV=hWI0V3$k8Oo@orAAwc!c+t9j0Nst7m{T~BtBkRuQiL@_!O8e#ywKW~ z3AH~jUwRy1myG;f!iVDYILu`g0=W{H@1Sf**kAR?MjcusQ6xA8oxKY3wB&14?A@!B z%SxZgORFi1&w4x22YGoN)f)l1sok52(}Ko8Mgm|?f~O)`Jx1rv2EapPLR~3!g52C` zmN|3CP8zqdP(KACKup2jlg;ja052HXyk}0Ln{n&;SyVkQ4%%5H*<1Ssa*0H|EfM<~ zmWlrE$NgZyvVtcQ6XJB14cJx+*0ZUE|329)u8GI=HAY?MBlfa_Fh@)Nk#={Wiz>x+h$eNYU^aaIaA zk>3p!^Plcn=yI>8F`l=~7g=b&JhPWE@{|mskH}hgXlzqXXbEL&0Zwo#de-0DSpU*x zNxeRt!nX74ZiFP1__eh=SauSWp@BRV#0C%tkWwPhlI*aWpmzmoMMIv~7~TmWhZu(O zqk{cL3OlDvG2dARFvjS?^XA*Y+w?F-pRZwEUB`GQO_p?t$FVO(3Jg`)IFtQf0W7Asc$=7YUb)K^TnF;tK}I}WCMz!1<|S+ z+scvM>qr0ueDy-vXZ3FNV$}nfLVWi#uWl{#m#K;?O~fg{YM>k;z;!#+P$+jRqI`S2FewtS#5 zY^g6S=8l?Cll*2%h%amS7##IjjP7oQvz)sXCG3r_C=b z4?UPYj;w%c;FCj8VT|P6=z`zB{L_ROLoqHgsJ!&mSC$2pRJZ29qNjrF!tJp?l~*?2LMhL zRRfiJ1GK*p^IhZnw3j+`(s1e+$g)IHEBDp4o6Zg1nW5s;i#G*i_!IXwp-MObu#p@{ z08AzUtOl7qm4^X%vuN@=Ap)H(w0S?}qOkCs^|I4*7(g@QfgByF)W3=|8vs^ZBmO^L zVrO~@^?S{n{m;qflAV_YhqCIrrvLW&tT9NFnHwmst-_CWhGyON>b7&CaceUL*|;w6 z(omZNd^@Q2sc(uNgS@$(bSnQY>kTmOPG$vqdL(jo0}QAJdsGo^{P zyA{Y&>P~5U^}Y9yX##;NpsA}GsR9`TEF^&hUuE@9QUSHEF}l5BK(03jL(j@xJ#0p!RnKZT6Ag0o8}cK2m0NqNnD{mz{%XpXwdKWf zITsE0{jbdDVF6ZcgobjrCMLcmq1@W0LiM>LC10IX;_CM5f~~qN%QaH7>h?8^*Z5m7 zQ*Zxz?Opig#)QTLl{pJFoU}UjRi!57yg+d)g>4?649L^#`-f<eBiL8;Riz`W6 zop0n`{YlXi<#Wv$kIgn`o?0POyV!{D8UX(RfFGTc_Z%uYLnB_O6YE!K7M=L2Ip>L6 z%{}qH#>h)EsCd>9%IxjlZZ`w`_pD7d^LVtGwso;^IBbYOagLI-kCjO)N!0m1xQq`P zI^vLVjzgC>(1ws81t7y|VbOb1uK@wuDpKmJ1*s8#OL!{pdG(dIR<*r1FY}DezUw^F z{=I_JrrxGNL>GKTe$6A9{QTt}iHff5W9W1utm2 zUw>T1?pDj&FCC&SH+Gsgc*pmF1DZh47|AFEao|~`1R)**)=QYKKzUW}R(Vzah~Akp zR$QAQGNdbIH@!OZ&FKKA+?Aho_K(Uk4X0f)tJ3u?S529*{9S-+jA2m^mZyH-trY_M z(Ws_xv~9huDjA{RbQj3c02PK&jRqfdttIA;Z#rDL1A9=+ev(CCN}{fnb3M7Ijm9V*h$t%8jzg`Z|DiY ziNtTs{YX6lj6|8xs>2}%zeP2us_#)js1k&5>rK=l-D{;(5s7;wO5#`r6oFJNf4G+@ zPjO}15s<(Rf7@m4l6CFF1NE4q0gAAd^paC{J{=ZkrUazb>6c^!I^OViUtGL-g%tAi z1o{*ZHt-(sDTqj7>>(iC9kvvFp5Y6pR90k^X^Hh$7ae+ck4@5CCRVj{FA;L5WL~f$Ic{bEF3BtSxnFiAZr!0@YJ>3Q`KUK`BTaB$zJ%;fCr^5}}P6 zFL$Do?)U|Y+ecT(c|%@L`u>#uM+`1uSCuv0J8tcn`@iu-RK@MN6dM48Wyz`*4X8@0D^Zz{Noq|d*-oOCo))5xl&x)p zC09bdu*Jd7*Yr`dM61s_r0AB`+MTl+M(v5=*`;Bhbr^oy3igJ8XJ( zSC-pPZ)v){df4ppM$ZnoDh)BOft_mdc0Z~Ud2UAGygQprx?H5~KOk+T*7y~`Nf^!| z31Ddld4sQ{)$YviBHkCzU*0NDkuu{jt|1B1@{6jw3Z@r7;~kQtsLnwm-{>9OP52@g>_-GGS+lhRSQ% zVvDi~_1X55#(1JKMTT-W*D~Y|w*xucdDr!!)bO%yI!m?A?i`%ZB}eRDp4>GLp|@t3 z3wm8+47*>)&IQ>@xQTvIf!r;S-)Y3>%`oTAtYvg2LeDM;S|ucz%a{vq8`L2)WV&6FHy3`vx<+t%Ngck_TU8QCDuPkK1;E`vDU#+0{v7j1 z=bZFWmw$CylR8!r_LV>zfy{I}BujxU3eG2u*!S#2bIJCOcVEY%Ds!?xM^FXyr%cY{wa$+$43+&9S^(W*QVZAI(#&0cQ0$G7G6mll;Ybl>3@ zP)-V90J$J<0ZKt+1OEj69!{_NOkP?N?Q4uk@nSbpuzjwLAtN55rPIy%6#(OmUQABP zI;}ewmlgDK!(+W9*JO-nqGP2Cq$Y4;8g|8l*6Ss0Uw5U@4Z*IwQ1s!?76w0ej(?dw z{41gS{3a9@?xn9W{G$}&m&qjI5TLx*%jV*@j}VM-4p|MB))$sxCr&xnlPAUIN|oG-i2oa5u_|{(D!j1VE>Fs>*MYj$DHEZMgW3;1NC+HS0NEH|wX2zhoXAxOxO{J>wp)B&t?N4k??8?^_uct`Zvh?%w&&rZ zZyUV7@wH#DZ_YtrTs~|xAI*5_P-<5NP67&nslZtvD?s@tQj~@IY_@wE!%shfiWX_k zjt^c)`%H>yFu)i#B!j)HrkiiAYTOO>eelc2d8u(LOU;S&G{$vo#2)#YIe+!h@~(?< zS4s}sdK8*P^Toegf_-Q{OBdsJ&960vZtxQMOdtzm_&y(E44isz-f4Q5;61gjKPP_z z)AL%v-%`ryrG-$Y9fUAs~>Xn^s z{Fm00J^v$7=8ZCXTo2Iu2qokZ0ql(6gQ+L7_hTLF6a{O#gF4AU#B7ufTa;EQpA>y0bDUll&hh*?`$UGxCA&)h*zV_bTkk>7ThG4Moy!70xoTx{@y`SC&ZSmdh5 zKEa_iWebZ6YL553@)$3wlcAktgwZd3y14+u7(lT(jjqP6DJu+=%Z%|C>&SGF`$5i; zh(!{y$HsHcPSJ8Hz$@l_0IoHLwaGW~PL+KTFBH84KsV0=hE6=!NqSnSk1;kgXymsY zYFty>`zxi5Ae4|~84|Vu4(Z4q#+bW3SRNa-jV7gfS#VsspYQ~MR4mk*x<8rq*97lI z`9>IheE?4teZ={BBeBQk^KQ{|0E`Uwb+bW6%LKN)5#S3QYsw4eCm|12j|!ZN1^pT#aywvpE3OGbSlHA zIy=7q^Q=$h0As{M9^!o!9LGa+()3mHUJ~Czt}*8F0FnPua9V(j_+{uX1x0U6%{7Ld zmxA>VkkcgUCZJarDh9_T>U6nnKm1)67Nh_A$l`Od<4S zEBQs|zZb3F*~$1rc561h;iGE)Ub6ON5S=K9zh|1S_h@?U)5?YV%By_DUsaHe(%8{; zDvOq1%Z>A^vo%z#AEt(9cMtkU<@<;}s^GLzaK5}$e%WoI=-Mfj&@&1$OCT$Kg#VP| zY@YSR=u$v50%vj}Cn_Dd*9t_}#Qu9$d{FxHJ5-#e0Im4$%&)oNyi6)zaaXy$y3m}} z5ujD@NzZv=<>MOGr3zAbiJF4bAMdNaNZxA)=x2-?<0E>5K>9sI-U4Ad_#VcMZ8E6%tBQSwK*mz3`0G@&crL&|W7w-cB28m> z<2teCC&|yd{va-$oW_CAUojUw;hI9j@AeYU2dVjG=%s?9H=c4=+csrj>V1=Rtj-Ef zWejDx2diZ&`)-_Keo%HqyGV(?9|(hJTu$p%`9}6ve!1_Cmx>mD?XK`qulk4#kr0;% z)k1q_WXFC>b2_yDL1jm-hSlCd`IkhwBrcRYfT;>vKY?`xh${q+4JdIu7mPIeg$m3! z{^}9@Q=(+5*b;CgiW(x&?eBT5vfjta26Fa@HiKdPGbgcSqJlijL;MLHx&wHKuLIaK zXM*#iG3Jv~h;IZMmEmvFmAfv{uzLyQcM_$!jn;LnzgK@9%TSQ55?L(mqu7>4e5i)q z(nk9ZoQvX!p<6}MIZ)~#q=O?O2u=jbfT~#6-I3mt3P!3pCnQ`TVlUa9QwTqiq+2za zix8Poo@WpLO2=x4%R424_qgze~YU9F$Li zxP#aa(2TvCmN;F;0=A0v`D4!P)B@2T4R+e7do<#EY*eWtcLzko^`E&(JQ3VyB`bhv zn>$7Xd_)Y_CFIPNKouzyJy@9zEOUW`DvFVP>`||Hh_%xR-CAT8ddl!$Wup?sKmHu! zcPTpYTXkY4jlb_J=+6zoQl z__FTC?Yh9}B#14UVa~m;$ej621-?0n2uwvAvFAcJFD_c}syX+Sx#o<|qG)dds4}`| z8_j+IHpTIE&Jq>Rdx-9A?(80NxiLN)WD`Jo*}bBA9I3=_Q;;WniF~w;=N7?NZw)INcRur54IV4hq8c@^6a~o$N|ulnaa=t^LIMgZ%DQbAh`>$6n(gA|Ur= z5pS$OYn_T`x)ft8#5{niW3Rf@xUoG5ze4m|S6WBT1)?h9WhWcsvc1GE)Uc;bHy14d zun_B9%T4a{xC>DAaA5wzqG=nSELvKUF3PVG*o5(I5+ZWGh9$g&9(U)%Qd~fX*C@m< z6iCu3ot&_@(p(nA*B+6PztbGYPTHKRW4&J@PTY3A(er#i@)QL!fvBH~Pc`gIfk+Db z*SMZs>uLcTeX?q=2j>qu_U0@qpRbVWCm`|QEQ_$uuRyknIC)1N$|c6{Psk*E4`BPL znx`QE=h9%u|-;wN&CM!c+z+|b{_-$%tsZAAIQJDiNSC4@9$Ujz7vcLEfR*m4FJ zr1^Z6a$eek^4hU4uo_mMyp6qDxv#Bq*Ri~Aofx9 zatSAVD?us<^&|r6Za7n<-)vmg#s!MZ8sQJziHo-fNTKY5RKgvB z6pi>2mrqVNpu8zN)(5cD6S>W2j5n@K?Ge1Skq`SG4U2Nty{KTHEwGtdvAo7XqGDeu zkn6H2E%g)Wq2f4E0=wE66I6lSUSRX0{n5LM`z!*bG=6HZaor0URQ(-9Oeg-$5$_@; zp{Rr^2i03n*2+xEKUQ$kyhJ9xxc9@=UhEu!UG|*$*(P_e@)Cjb{qsd1ha?=ZP+RXI zYhF_knxSGx1ZlCglbd;VjrC< zA{x=VG~%UA)mn4qZX6*9n!MOh^}q zb9ADc_1RbvltLTjV+Yl*;Peql0n*Nt+rK%YP*IEEqcjsMw?$-nS0C}~RGiid_GSfp z{wwDEkIBDHv88x0jmlNYE+w7#7cK}=dU`0o%S-f&DBk;ZB3V<-H#hb+nvWMaF$?8S z3d;Ekc9n)T=PKqb{>b=qn$EftRpOxt{#!Kcek#^RFVR;E%;K+l8`t+wr=q<@^jZb0 zGL=oY^)HlmLKUn8c_(hwiRkD600PQML_t)J77ES*iCpb>%2!R9^;Q{xCDTkxl(zOD znrT=?7D~34%GrrX(ligz1y7k@{Q$QUGR9n!LTIdq*oRekAJ+)m?O3+HPKHJ~Pzmg0 zT`C~1*Ra-Y7p)^<(srd0z9WUuDq%uvBjej_4&3G=wC1v+l2{ibXKZt}e4ydPG@>U!_w?qQ5>KiC+uXNb}N?Rq!%kxHJCFM~$DGz>$IHLkc6U`((b zJr3~-^ijdSKWm)1#^{49#JHME@+LHR-7uqXwI^Q$igQu}#W~{x#W`yN#W@!RigRif z@JORyt@{?_HKV^VqUQIw;1yhvzI?Q<*hvg(aH*Ekr<8eOOVy^S%c*BHas2NV3So`V~V%R2#_+WV+dQ;se5 zH-_&>qw3w$`11OP%uTV{aj`KMr4oK4o&9%DGvBHgtn6oux+6gNQXf^53e1J?jxnxe zn)&{}eGG{-952ur7=q+sgxm>2HuUt(|L@e+_?>9W{yu)<69h6*r(>-m gf%<{907*qoM6N<$g7(UE!~g&Q literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/snappydata/snappydata-310X50.png b/core/src/main/resources/org/apache/spark/ui/static/snappydata/snappydata-310X50.png new file mode 100644 index 0000000000000000000000000000000000000000..39b94b00aba45db1905715ed83a6cd33825be7c0 GIT binary patch literal 7750 zcma)>^;Z<$)5kY}rBhP6yF;YAap^9Rly1pgS|p^qyFqG^T2MieQt6NdSr8Cd8l=8{ zp1qU4U7lCQNUxQRq& z%3iOqr{aidCBI&g-D7vP#>w`APM|<^6qHI-lu*S-<%PBlhu3$SZR!Em~x zjr!Wcy@s2?%ZTZlh@qYSog#wvkK{W1rJB8aAEzE7(TgF;&tG%oq$O+CMim31T_s&#zrUD--T{PECz9e>G7rX4;um;Do3ZF1fH~2o*35 zYIR8t@aG!Ne)h5Q@qcx>Gd^FhpSjmH$MlRTp(9-YxuCIVW5A^1GE%^Cs0U<%Btx*4kZIrD-jsA#1E2|~z+AvOSd#Q&j^{?qaWl_me!ubA(bdlhip`~D zjvRAP^1+utA2i|-D_3+f5ztC{uXL0L$j!!A&WR+KycYb`Xdw4a_SMq z*!!ixQW8yheg>%LkFWAIxdVL!pmNf1Y8A@Bs>YSy|Q@k_~kI}}3_i9&>-IX^Q++ES)*)+44VB)q~|H*@QjgM-vpkRSzxWVXChDsv>1*+6f4kj0)SuASVPA+W#6XIcine)CQ!hldo)!xR_%e~)$_Q%q zb)@h+o_1r0fHyeodB!Oeb!?$zg58rI=3Ge5(VsowBS0$~KbT5mDgonZ-_U+91z#bJ zO!;p9c3^9M+yO~+VFc&FO(+lVh!F7Q#n_;rPvbSP&1}IKV4Gf?3344y4J{Fq^A#T# zBH#ZG!&qcld89b+{MEsi;$VMer2eD%m3jb^NsDN%(GX+Hin_h@NEi1r^`AQApM9SC z40J8vXYmyQ{pz`XW)3>vj$9Ts>6W%h2~($ka8;1ScwEXPNCzH?_#0C0F~u~bvEPsX z;p!Ic2z?^IP_XHSdcOiw03U4^^snV$9lqTpAoCEISUs%hJ~WdP))BO9aNZrBl)RIDhym<3=p`JE(ab(HxQCs;U}2emxB?|ACV9*GzJMlHIqQbDpeN^E?O7f>v+|5(p1I7mctW01@D8_ok{ zHSHwi%+}(q;Cjkw%8hjLT(z=o{7C>Ljw(aG2yfNn@3PSa_xv`9`-W<{V9N3P!jki2 ztQBpP8_GJ1VNFEsG@m4l&3ZW7vDIOo!U`Ve8P!ZXBqg6qH%DAp5#d=o34N*+0Cbm3 zEZ`5_s-C3w%^3;y5TVRiJc9fc$G<8USkzD%csd>=NEF_R_XH0OLb=%Ryd4>aA5Y%X zn`TYDVT2^73WF(We*Y>ZT3yuFeaf&4oR2huAMhRUyb|aZdms>|-|)){l9~{v z=B__Rb7}&)*kjk7vplg6KiV=5Xa|AZm3%JY(_k|Yb*%0qp=X6(a7#^s^lxWCEdW8# zE9?N^_$jUc%aB1(dDP0;O))Wm)l)xF1i^gKmYOKmc-N!C-&E^(%7d!toKYyZ&MgPb ze`g9|BAsgQo$6sJjm2UrMe;OKINA|#20tlb4<4$nfvxQKFh|B~t1^<;Azs<6BjSRP{-Eoc$hI$=js7aSSnZ1p& zWn^&sWC73sw{}fx+{g{VgehGzE(p5nkuHP*F<$A-NWRhc?Bd-7a#xO=3(xWo_Uk3R z^YuT^VTmS-UMg$B^-^fFBCO$!09sFb2IG%CSiRQYoZx+H&xwD^KvNa-94QXKQL3*H zA-dJA`8u^bODugt`GkpP*pXd4ZjQK!J7oAFut@WPCHVOM@8N{^2(d)V0TPjwj7-Pj zWm5?i2Lx;1C{iQazzB#FFazwd$?rL3EXAbseGa7C`Kd!|Lon+bt{nj^k#C#X`VX6q zmWe?5D~%_Tr-b{CMhRWzx|J^_IgrX(g(?xjW|)3>odQuF6~ZCriglLUn;x6OdC1J_VubV<>i@Nj= zMMLGwelCnetX!X~3xa{hICgLdsi*~lDg4aqR7JQLdwQs}hMN|M)vv844E=)mh;fh2 zM7}HfR7He5Etj-}M~!}39z~k+_q@)l8%(7Cwp$+4CMUk??@#=_;p#N3l@XmQVfSvEyJ>S zJW>Fhte8&*h*!FJur1ds(?w0!-M5T6oc8iXN{`Qyh7IR!K3ajv#7t`;+Uj&urE70T zkODM0Ka%~3;r5Gbyx-x)_-*5V|5_%xa{GwHV?^301-;`Y6axnz2OhhOposCy+f4m+A$zy6qq)~17dq(COYOQp=@XU(Rmr0A` zJBz2>N@ozVdwaX4fn%mozAmWrP|?B*w$#bX3y#9hBIDA<}-*IR zUYrPUoLJR_+}iIemcEm@jliAxf#eRyR@d4gx)_y89@@M>J)9J}tsDb1}0zz%@c>UF5vWVS>$iNt35dd>Z{LT^7kn|K`o5 z2j1FEKfuaXzom6mFA7!x8O6Sx!KL;0INwFlpQe=Wmx0NuGou5OnCO z)-%L3gvUCSoiFqwjb>hK^{{{;^)CW#Y%-Vne@z=kZ=@__c^_6 zApVr^VFJxOZ+nW4|I(7|s@6;1Xok6XNycf|1E)z&cEL>d2QzS;*RWGIyNXAs-RsXU zFtNOjFNeb}VGGb4I*y;un*_2yEZh5yx-1@!h);jIj$oNhGqoQ5R5-s7Y7I|8;fC>s zbDRxm&d=EJj8#{zdPzPEy;aB|Ldjcqo{F#HnHv-8;@4?}kfI4uSGYLY*h3O!X-oQa z@|KbHR-73}-f9S(;EDx>&V47HvYzg18`KMwxV^+k2O>G9Hp#V%e4`ZsjavjMx%5`4^{rm7h0Q03a2IVCG+ zM*S*$VMBjECW5XBo}y4lzSjra(kef0Rb{ZE2p%@gav__i12{YGgtF+W<#+J`1!fk6 zz8%kYe_8aGj1Gm2|G?E)bS}F4V;?J2xoYHYuamE^w=+9GXXO7J|2tEjqY#Z-G12P4 z?*ZQcF1pkO0gQ#L<=uE1|)lBeQg8B!Oj#Z?%04&V29&L#s022Xa%{ZtilUH2P zPLl-JvITGoQt38ayTUXb#67D|C@Qc)1jF5ZSSgG3$S_ab=~7(U=Rjz4aXVcyGXHJg z7EzYW=MOkwiclFrr^~j1EXp~rwGOsX3(~XpX!4)zrz%zx zDKfO!!>9o{gSZdgxxKT?H&4HX4;J*rv$GCaN#)tbM#Mvyi=e!toX(@SkdA#n|2=0R z5A^tP)JI3T@f2nmviz|>ahE}e@jrjf@V78cghtTYwK^PDkod`8X1@2-Wr|`O z(Knv@ke!>aaxg+of0d#&V;m{+8}PWwCGQ-g0!dv-Za6!H`7x{YToIb!bUj;S}(yW?d3G!|J@R><9MgAU=yBol3$u9+ExB z#b-Z0Z`|JJnQgbzO?Ak}7M~XM`z16B=_f+hZmUe_yjh-bKy#T=?V)DO|1EB3!)w27 zpKXeg`-O+^(f^@!TnBOh8&n^_U@cc!I367qjKN3bD64Om^EGp^(SL+ zx&j5wFV@GZTzN~u^x(~|{6Gx!i}T3z6*4I6U^^25=7{^eqc{}(rk}XBsS`9-ucO2# z+cnNXD^aj}ohVXkzjZ5KajJLvg%rH$gYk?)t?IXA?jpBhg@6z66ZO|iiQf00th7C5 zodn{uk-FUDN4?l)zE121ER+^ZYyR-w`&b-cJi- zFvWjZMsiJhdpcKt^ooS<2+2OJSi@?$H5t4B9AJ&;r6B`oU@!|w9-k?r?C;3yz zSR|Z8#6F0k=Kwd{&Tgf-J2Hk_PwJ(;hEjbPEPbkp6sB*U-*;-j3txw)3 zmWyUCQPCxD;`D}${CZ>*^|--;6~B~pXNP|!EI4f&xicwN?FaE}{69n^K`<{DWYZb5Pwa}YQ zHNNif?V6M=tYU+DRXUFZ*EO$`1G*exU)>b48$i9;v=l;Wx4C&HM_KOb=g_azk5%gZ zd2RISc;CUJ_ky&iPca|vIi|r19oW--Sm(KMoa0iZ=EpizkJ82t*!QDlA=EZnrGP^I zIC1cLn|z?jIU8_RrqoRz<LJA}t9B8vFwG7pI6yWUi(0F4*6&eZXSeb$!e;_VN372r<9@ss7oa(P0^1Ieh082%8pc6qn`nc?%DhaxL2 zlwz&k3H=Vw3ig0StWKi?Pry0IF3Z;*^OG>#3U>kBT~1D4nfR9kS9=KMbwrw>bl5OM z+;0?W7OQP09!j{mHu15=)!VxI%I{DbvFUrn&UDJ0&K0;R1i8$6)xV`+T-w;~co@+U z*Hy7MJqc&kc|=bbXg-MhR8@=t8)G64Qh6-3!sLW@%L2w1$PQwJ9uv)0>`6Mo%DERr z95)B*^d(&EjgLQwX3=T>Ns!T^RT;zDF#NngHdQ8PjW$NPGCfGfhNASuA2|sUj!;&6 z5c7-&**lWV-OIhHf9yUxQ2mznnKG|H5&6lt-hF8bP76g6!);lT?HozAq6C=%PXT~q zaB=#)n$NZd{p>|&a<*}5s!Wa?q1p3W@u#MwS{vgGYiZvTxdgpO@*4$M$`-Nub;nMW z44xk+Nw;LiUbm_@o`m;|XCy?t9I5rGz9>{V==U0&Q6|Vvt(B`0*yqe=9&v2&81K&g zv%M9t?c@q9_TXM!#Ac(3-=joO)aK|<+EM#w{!}ZA7ud8Po^URzywe!n%wg@<53|%) zur`S{QYMg#)0td0k9NYC#oD3hQFr*qhc9Bk1PM7z^g+-rYRIu_gga7ceZwc#DCqPH zN+E+=ur?dI}qcgS0f{}0Pjj??T zq(LSyd{Bez^B_qEDThCIOPQ;GB7ABxM_#C{RiWd2k>0%4aW1_?Z)xzbp}1=%0Z~P8 zmWj<}6+~ua9?!RLMETojbeZ8rpB`(~+g;r5oS^&a*#c5i9mb1qaaofI^9I|s$qmNz z{p$HnBxfY;TbJ;4YK=E9+ACa8Aa|=3DFMM~di(wD2+D~mdBSw^a}CzB5IgH4yIFN6 zlcYiV5CZg`pLyQP?^izvlSP3zowoMv&JSjb2GeV}I~i6I>~VZN0nBk@2WZ+-UGG&# zP_(2LZJRDCzoN;;?CW8vTdP2>z)k?UAR<`i*3slrb7QZ^;o8%`kb!<&mP?Zc1 zj1d`SxCo<}9f!v;Usec2a=#Y#c<;incWv?vtFvp)AKsxOV__~3^eJZ2W3rj5LN=Ak zWRBV-0!YnCCMXcV%&H>eaK!ovLLS_Gtkzs&6DC-nKFtG7@(q1yH4|i>R}Y<4TcZS@d=5c_EQeY~TFa&WGaSwD7Q$8C zp=-11A66*u$gKO*sR6@gRbog3rx|8UcCbLgQ8TQUIcUuZE-6`XZAWF_=C=AHKFR0J zlP*#hzd>48F+2}7z6Gl=?+tyfFl-tXZJv?N-}LnFEC8<+Z4^|%M z#eQbiK7CQJ#*tli30yF;8pcx3*PWS0T2 zpLHZypsa(YJLfJC%<63wrUIij7$j)6Kzov15>5|pOD#vAuvz2l@=X2cUOp%J#87d| zG){@r-Djx%%}WeJCpfnYC_FX($j9TA?gcPS^i#l4#M*(sN?3XISIWa>2dA$8|QO6*8$CPRLPA*x`N zK1)x|5o26~-@b9CMz;JOgV#GLHOCd;L9Ohxxpb7{us)YWGgeGZ?5BNQSZVl7XKbe$ zvaU4!)RQAiE-Rj8qG>Z1$7WbkMS%i9dbyQf@J5ZfGRb@Ta5r9o z6Bt>;FMLP)s8n{?Lv63a-0sug0#miVxrL;`nSQyOyh)GVz%g`F>jBc#NDh#ve{ZP7 zq*Pik1!<}59?UoYPv9g@CMt8@yhlY51&dx8q`uOHH355Lmg7>YU%3CL+bJYjOXJCC qTwiJF1uRYKyXc^#1^HkioD3 literal 0 HcmV?d00001 diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 935d9b1aec615..ac2c8e5102543 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -22,7 +22,16 @@ } .navbar .brand { - margin-right: 20px; + float: right; + margin-right: 5px; + margin-bottom: 0; + margin-top: 0; + margin-left: 10px; + padding: 0; +} +.product-brand { + float: left; + margin-right: 10px; margin-bottom: 0; margin-top: 0; margin-left: 10px; @@ -251,4 +260,69 @@ a.expandbutton { .table-cell-width-limited td { max-width: 600px; -} \ No newline at end of file +} + + +/* SnappyData */ +/* Popup container */ +.popup { + position: relative; + display: inline-block; + cursor: pointer; + -webkit-user-select: none; + -moz-user-select: none; + -ms-user-select: none; + user-select: none; + line-height: 2.5; + vertical-align: bottom; +} + +/* The actual popup */ +.popup .popuptext { + visibility: hidden; + width: 400px; + background-color: #CCCCCC; + color: #202020; + text-align: left; + text-shadow: none; + border-radius: 5px; + padding: 10px; + position: absolute; + z-index: 1; + right:0%; + font-size: 13px; + line-height: normal; + margin: 5px -10px 0px 0px; + /* bottom: 125%; */ +} + +/* Popup Arrow */ +.popup .popuptext::after { + content: ""; + position: absolute; + bottom: 100%; + left:90%; + border-width: 10px; + border-style: solid; + border-color: transparent transparent #CCCCCC transparent; + /* left: 50%; + margin-left: -5px;*/ +} + +/* Toggle the popup */ +.popup .show { + visibility: visible; + -webkit-animation: fadeIn 1s; + animation: fadeIn 1s; +} + +/* Add animation (fade in the popup) */ +@-webkit-keyframes fadeIn { + from {opacity: 0;} + to {opacity: 1;} +} + +@keyframes fadeIn { + from {opacity: 0;} + to {opacity:1 ;} +} diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.js b/core/src/main/resources/org/apache/spark/ui/static/webui.js index 0fa1fcf25f8b9..12d8d7d36f6cc 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.js +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.js @@ -50,4 +50,12 @@ function collapseTable(thisName, table){ // to remember if it's collapsed on each page reload $(function() { collapseTablePageLoad('collapse-aggregated-metrics','aggregated-metrics'); -}); \ No newline at end of file +}); + +/* SnappyData */ + +// When the user clicks on div, open the popup +function displayVersionDetails() { + var popup = document.getElementById("sdVersionDetails"); + popup.classList.toggle("show"); +} diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index a4f8f2b798b01..3c662e2bf07f3 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -21,6 +21,7 @@ import java.util.{Date, List => JList, ServiceLoader} import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap import org.apache.spark.{JobExecutionStatus, SecurityManager, SparkConf, SparkContext} import org.apache.spark.internal.Logging @@ -160,14 +161,14 @@ private[spark] object SparkUI { val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" val DEFAULT_POOL_NAME = "default" - var productVersion: String = new String() + var productVersion: HashMap[String, String] = HashMap.empty[String, String] - def getProductVersion: String = { + def getProductVersion: HashMap[String, String] = { productVersion } - def setProductVersion(version: String): Unit = { - productVersion = version + def setProductVersion(versionDetails: HashMap[String, String]): Unit = { + productVersion = versionDetails } def getUIPort(conf: SparkConf): Int = { diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 0ea856466a507..95ff953b5969f 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -230,16 +230,27 @@ private[spark] object UIUtils extends Logging {