Skip to content

Merged Apache branch-1.6 #174

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 41 commits into from
Sep 12, 2016
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
41 commits
Select commit Hold shift + click to select a range
9808735
[SPARK-16514][SQL] Fix various regex codegen bugs
ericl Jul 13, 2016
d1c992f
[SPARK-16488] Fix codegen variable namespace collision in pmod and pa…
sameeragarwal Jul 12, 2016
7c8a399
[SPARK-16489][SQL] Guard against variable reuse mistakes in expressio…
rxin Jul 12, 2016
980db2b
[HOTFIX] Fix build break.
rxin Jul 13, 2016
fb09336
[SPARK-16375][WEB UI] Fixed misassigned var: numCompletedTasks was as…
ajbozarth Jul 13, 2016
4381e21
[SPARK-16440][MLLIB] Undeleted broadcast variables in Word2Vec causin…
srowen Jul 13, 2016
6ea7d4b
[SPARK-16313][SQL][BRANCH-1.6] Spark should not silently drop excepti…
yhuai Jul 14, 2016
f6e0c17
[SPARK-16440][MLLIB] Destroy broadcasted variables even on driver
Jul 20, 2016
4ff9892
[MINOR][ML] Fix some mistake in LinearRegression formula.
yanboliang Jul 27, 2016
15abbf9
[SPARK-16664][SQL] Fix persist call on Data frames with more than 200…
Jul 29, 2016
b6f6075
[SPARK-16751] Upgrade derby to 10.12.1.1
a-roberts Jul 29, 2016
f445cce
Revert "[SPARK-16664][SQL] Fix persist call on Data frames with more …
srowen Jul 29, 2016
03913af
[SPARK-16751][HOTFIX] Also update hadoop-1 deps file to reflect derby…
srowen Jul 29, 2016
1b2e6f6
[SPARK-16664][SQL] Fix persist call on Data frames with more than 200…
Jul 29, 2016
8a22275
[SPARK-15541] Casting ConcurrentHashMap to ConcurrentMap (master branch)
Aug 2, 2016
797e758
[SPARK-15541] Casting ConcurrentHashMap to ConcurrentMap (branch-1.6)
Aug 2, 2016
92ee6fb
[SPARK-16831][PYTHON] Fixed bug in CrossValidator.avgMetrics
pkch Aug 3, 2016
52d8837
[SPARK-16796][WEB UI] Visible passwords on Spark environment page
Devian-ua Aug 2, 2016
d2518ac
[SPARK-16873][CORE] Fix SpillReader NPE when spillFile has no data
sharkdtu Aug 4, 2016
c162886
[SPARK-16925] Master should call schedule() after all executor exit e…
JoshRosen Aug 7, 2016
1a5e762
[SPARK-16409][SQL] regexp_extract with optional groups causes NPE
srowen Aug 7, 2016
a3b06ae
[SPARK-16939][SQL] Fix build error by using `Tuple1` explicitly in St…
dongjoon-hyun Aug 7, 2016
ace458f
[SPARK-16956] Make ApplicationState.MAX_NUM_RETRY configurable
JoshRosen Aug 9, 2016
b3ecff6
Revert "[SPARK-16831][PYTHON] Fixed bug in CrossValidator.avgMetrics"
srowen Aug 11, 2016
909231d
[SPARK-17003][BUILD][BRANCH-1.6] release-build.sh is missing hive-thr…
yhuai Aug 12, 2016
8a2b8fc
Change check for particular missing file message to accommodate the m…
srowen Aug 13, 2016
b54a586
[SPARK-17027][ML] Avoid integer overflow in PolynomialExpansion.getPo…
zero323 Aug 14, 2016
4d64c7f
Revert "[SPARK-17027][ML] Avoid integer overflow in PolynomialExpansi…
srowen Aug 14, 2016
5c34029
[SPARK-16656][SQL][BRANCH-1.6] Try to make CreateTableAsSelectSuite m…
yhuai Aug 16, 2016
60de30f
[SPARK-17102][SQL] bypass UserDefinedGenerator for json format check
cloud-fan Aug 17, 2016
412b0e8
[SPARK-17038][STREAMING] fix metrics retrieval source of 'lastReceive…
keypointt Aug 17, 2016
b84a92c
[SPARK-17316][CORE] Make CoarseGrainedSchedulerBackend.removeExecutor…
zsxwing Aug 31, 2016
d8a6426
Merge branch 'branch-1.6' of github.com:apache/spark into csd-1.6
markhamstra Sep 2, 2016
21be94b
[SPARK-15091][SPARKR] Fix warnings and a failure in SparkR test cases…
sun-rui May 3, 2016
958039a
[SPARK-11301][SQL] Fix case sensitivity for filter on partitioned col…
dongjoon-hyun Sep 6, 2016
e6480a6
[SPARK-17356][SQL][1.6] Fix out of memory issue when generating JSON …
clockfly Sep 6, 2016
af8e097
[SPARK-17378][BUILD] Upgrade snappy-java to 1.1.2.6
a-roberts Sep 6, 2016
3f797dd
[SPARK-17316][CORE] Fix the 'ask' type parameter in 'removeExecutor'
zsxwing Sep 6, 2016
69fa945
[SPARK-17378][HOTFIX] Upgrade snappy-java to 1.1.2.6 -- fix Hadoop 1 …
srowen Sep 7, 2016
047bc3f
[SPARK-17245][SQL][BRANCH-1.6] Do not rely on Hive's session state to…
yhuai Sep 7, 2016
5d74c76
Merge branch 'branch-1.6' of github.com:apache/spark into csd-1.6
markhamstra Sep 12, 2016
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion R/pkg/inst/tests/testthat/test_client.R
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ test_that("no package specified doesn't add packages flag", {
})

test_that("multiple packages don't produce a warning", {
expect_that(generateSparkSubmitArgs("", "", "", "", c("A", "B")), not(gives_warning()))
expect_warning(generateSparkSubmitArgs("", "", "", "", c("A", "B")), NA)
})

test_that("sparkJars sparkPackages as character vectors", {
Expand Down
2 changes: 1 addition & 1 deletion R/pkg/inst/tests/testthat/test_context.R
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,6 @@ test_that("sparkJars sparkPackages as comma-separated strings", {

# check normalizePath
f <- dir()[[1]]
expect_that(processSparkJars(f), not(gives_warning()))
expect_warning(processSparkJars(f), NA)
expect_match(processSparkJars(f), f)
})
12 changes: 7 additions & 5 deletions R/pkg/inst/tests/testthat/test_sparkSQL.R
Original file line number Diff line number Diff line change
Expand Up @@ -1119,9 +1119,9 @@ test_that("date functions on a DataFrame", {
c(as.POSIXlt("2012-12-13 21:34:00 UTC"), as.POSIXlt("2014-12-15 10:24:34 UTC")))
expect_equal(collect(select(df2, to_utc_timestamp(df2$b, "JST")))[, 1],
c(as.POSIXlt("2012-12-13 03:34:00 UTC"), as.POSIXlt("2014-12-14 16:24:34 UTC")))
expect_more_than(collect(select(df2, unix_timestamp()))[1, 1], 0)
expect_more_than(collect(select(df2, unix_timestamp(df2$b)))[1, 1], 0)
expect_more_than(collect(select(df2, unix_timestamp(lit("2015-01-01"), "yyyy-MM-dd")))[1, 1], 0)
expect_gt(collect(select(df2, unix_timestamp()))[1, 1], 0)
expect_gt(collect(select(df2, unix_timestamp(df2$b)))[1, 1], 0)
expect_gt(collect(select(df2, unix_timestamp(lit("2015-01-01"), "yyyy-MM-dd")))[1, 1], 0)

l3 <- list(list(a = 1000), list(a = -1000))
df3 <- createDataFrame(sqlContext, l3)
Expand Down Expand Up @@ -1389,15 +1389,14 @@ test_that("toJSON() returns an RDD of the correct values", {

test_that("showDF()", {
df <- read.json(sqlContext, jsonPath)
s <- capture.output(showDF(df))
expected <- paste("+----+-------+\n",
"| age| name|\n",
"+----+-------+\n",
"|null|Michael|\n",
"| 30| Andy|\n",
"| 19| Justin|\n",
"+----+-------+\n", sep = "")
expect_output(s, expected)
expect_output(showDF(df), expected)
})

test_that("isLocal()", {
Expand Down Expand Up @@ -1749,6 +1748,9 @@ test_that("Method as.data.frame as a synonym for collect()", {
expect_equal(as.data.frame(irisDF), collect(irisDF))
irisDF2 <- irisDF[irisDF$Species == "setosa", ]
expect_equal(as.data.frame(irisDF2), collect(irisDF2))

# Make sure as.data.frame in the R base package is not covered
expect_error(as.data.frame(c(1, 2)), NA)
})

test_that("attach() on a DataFrame", {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,4 @@ private[master] object ApplicationState extends Enumeration {
type ApplicationState = Value

val WAITING, RUNNING, FINISHED, FAILED, KILLED, UNKNOWN = Value

val MAX_NUM_RETRY = 10
}
22 changes: 12 additions & 10 deletions core/src/main/scala/org/apache/spark/deploy/master/Master.scala
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ private[deploy] class Master(
private val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200)
private val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15)
private val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE")
private val MAX_EXECUTOR_RETRIES = conf.getInt("spark.deploy.maxExecutorRetries", 10)

val workers = new HashSet[WorkerInfo]
val idToApp = new HashMap[String, ApplicationInfo]
Expand Down Expand Up @@ -284,19 +285,20 @@ private[deploy] class Master(

val normalExit = exitStatus == Some(0)
// Only retry certain number of times so we don't go into an infinite loop.
if (!normalExit) {
if (appInfo.incrementRetryCount() < ApplicationState.MAX_NUM_RETRY) {
schedule()
} else {
val execs = appInfo.executors.values
if (!execs.exists(_.state == ExecutorState.RUNNING)) {
logError(s"Application ${appInfo.desc.name} with ID ${appInfo.id} failed " +
s"${appInfo.retryCount} times; removing it")
removeApplication(appInfo, ApplicationState.FAILED)
}
// Important note: this code path is not exercised by tests, so be very careful when
// changing this `if` condition.
if (!normalExit
&& appInfo.incrementRetryCount() >= MAX_EXECUTOR_RETRIES
&& MAX_EXECUTOR_RETRIES >= 0) { // < 0 disables this application-killing path
val execs = appInfo.executors.values
if (!execs.exists(_.state == ExecutorState.RUNNING)) {
logError(s"Application ${appInfo.desc.name} with ID ${appInfo.id} failed " +
s"${appInfo.retryCount} times; removing it")
removeApplication(appInfo, ApplicationState.FAILED)
}
}
}
schedule()
}
case None =>
logWarning(s"Got status update for unknown executor $appId/$execId")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.rpc.netty

import java.util.concurrent.{ThreadPoolExecutor, ConcurrentHashMap, LinkedBlockingQueue, TimeUnit}
import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, LinkedBlockingQueue, ThreadPoolExecutor, TimeUnit}
import javax.annotation.concurrent.GuardedBy

import scala.collection.JavaConverters._
Expand All @@ -41,8 +41,10 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv) extends Logging {
val inbox = new Inbox(ref, endpoint)
}

private val endpoints = new ConcurrentHashMap[String, EndpointData]
private val endpointRefs = new ConcurrentHashMap[RpcEndpoint, RpcEndpointRef]
private val endpoints: ConcurrentMap[String, EndpointData] =
new ConcurrentHashMap[String, EndpointData]
private val endpointRefs: ConcurrentMap[RpcEndpoint, RpcEndpointRef] =
new ConcurrentHashMap[RpcEndpoint, RpcEndpointRef]

// Track the receivers whose inboxes may contain messages.
private val receivers = new LinkedBlockingQueue[EndpointData]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -364,14 +364,15 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
conf.getInt("spark.default.parallelism", math.max(totalCoreCount.get(), 2))
}

// Called by subclasses when notified of a lost worker
def removeExecutor(executorId: String, reason: ExecutorLossReason) {
try {
driverEndpoint.askWithRetry[Boolean](RemoveExecutor(executorId, reason))
} catch {
case e: Exception =>
throw new SparkException("Error notifying standalone scheduler's driver endpoint", e)
}
/**
* Called by subclasses when notified of a lost worker. It just fires the message and returns
* at once.
*/
protected def removeExecutor(executorId: String, reason: ExecutorLossReason): Unit = {
// Only log the failure since we don't care about the result.
driverEndpoint.ask[Boolean](RemoveExecutor(executorId, reason)).onFailure { case t =>
logError(t.getMessage, t)
}(ThreadUtils.sameThread)
}

def sufficientResourcesRegistered(): Boolean = true
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ private[v1] object AllJobsResource {
numTasks = job.numTasks,
numActiveTasks = job.numActiveTasks,
numCompletedTasks = job.numCompletedTasks,
numSkippedTasks = job.numCompletedTasks,
numSkippedTasks = job.numSkippedTasks,
numFailedTasks = job.numFailedTasks,
numActiveStages = job.numActiveStages,
numCompletedStages = job.completedStageIndices.size,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,15 @@ import org.apache.spark.ui.{UIUtils, WebUIPage}
private[ui] class EnvironmentPage(parent: EnvironmentTab) extends WebUIPage("") {
private val listener = parent.listener

private def removePass(kv: (String, String)): (String, String) = {
if (kv._1.toLowerCase.contains("password")) (kv._1, "******") else kv
}

def render(request: HttpServletRequest): Seq[Node] = {
val runtimeInformationTable = UIUtils.listingTable(
propertyHeader, jvmRow, listener.jvmInformation, fixedWidth = true)
val sparkPropertiesTable = UIUtils.listingTable(
propertyHeader, propertyRow, listener.sparkProperties, fixedWidth = true)
propertyHeader, propertyRow, listener.sparkProperties.map(removePass), fixedWidth = true)
val systemPropertiesTable = UIUtils.listingTable(
propertyHeader, propertyRow, listener.systemProperties, fixedWidth = true)
val classpathEntriesTable = UIUtils.listingTable(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -592,7 +592,9 @@ private[spark] class ExternalSorter[K, V, C](
val ds = deserializeStream
deserializeStream = null
fileStream = null
ds.close()
if (ds != null) {
ds.close()
}
// NOTE: We don't do file.delete() here because that is done in ExternalSorter.stop().
// This should also be fixed in ExternalAppendOnlyMap.
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@
"numTasks" : 8,
"numActiveTasks" : 0,
"numCompletedTasks" : 8,
"numSkippedTasks" : 8,
"numSkippedTasks" : 0,
"numFailedTasks" : 0,
"numActiveStages" : 0,
"numCompletedStages" : 1,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@
"numTasks" : 8,
"numActiveTasks" : 0,
"numCompletedTasks" : 8,
"numSkippedTasks" : 8,
"numSkippedTasks" : 0,
"numFailedTasks" : 0,
"numActiveStages" : 0,
"numCompletedStages" : 1,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@
"numTasks" : 8,
"numActiveTasks" : 0,
"numCompletedTasks" : 8,
"numSkippedTasks" : 8,
"numSkippedTasks" : 0,
"numFailedTasks" : 0,
"numActiveStages" : 0,
"numCompletedStages" : 1,
Expand All @@ -20,7 +20,7 @@
"numTasks" : 16,
"numActiveTasks" : 0,
"numCompletedTasks" : 15,
"numSkippedTasks" : 15,
"numSkippedTasks" : 0,
"numFailedTasks" : 1,
"numActiveStages" : 0,
"numCompletedStages" : 1,
Expand All @@ -34,7 +34,7 @@
"numTasks" : 8,
"numActiveTasks" : 0,
"numCompletedTasks" : 8,
"numSkippedTasks" : 8,
"numSkippedTasks" : 0,
"numFailedTasks" : 0,
"numActiveStages" : 0,
"numCompletedStages" : 1,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@
"numTasks" : 8,
"numActiveTasks" : 0,
"numCompletedTasks" : 8,
"numSkippedTasks" : 8,
"numSkippedTasks" : 0,
"numFailedTasks" : 0,
"numActiveStages" : 0,
"numCompletedStages" : 1,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@
"numTasks" : 8,
"numActiveTasks" : 0,
"numCompletedTasks" : 8,
"numSkippedTasks" : 8,
"numSkippedTasks" : 0,
"numFailedTasks" : 0,
"numActiveStages" : 0,
"numCompletedStages" : 1,
Expand All @@ -20,7 +20,7 @@
"numTasks" : 16,
"numActiveTasks" : 0,
"numCompletedTasks" : 15,
"numSkippedTasks" : 15,
"numSkippedTasks" : 0,
"numFailedTasks" : 1,
"numActiveStages" : 0,
"numCompletedStages" : 1,
Expand All @@ -34,7 +34,7 @@
"numTasks" : 8,
"numActiveTasks" : 0,
"numCompletedTasks" : 8,
"numSkippedTasks" : 8,
"numSkippedTasks" : 0,
"numFailedTasks" : 0,
"numActiveStages" : 0,
"numCompletedStages" : 1,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@
"numTasks" : 8,
"numActiveTasks" : 0,
"numCompletedTasks" : 8,
"numSkippedTasks" : 8,
"numSkippedTasks" : 0,
"numFailedTasks" : 0,
"numActiveStages" : 0,
"numCompletedStages" : 1,
Expand All @@ -20,7 +20,7 @@
"numTasks" : 8,
"numActiveTasks" : 0,
"numCompletedTasks" : 8,
"numSkippedTasks" : 8,
"numSkippedTasks" : 0,
"numFailedTasks" : 0,
"numActiveStages" : 0,
"numCompletedStages" : 1,
Expand Down
15 changes: 15 additions & 0 deletions core/src/test/scala/org/apache/spark/DistributedSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,21 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex
}
}

test("repeatedly failing task that crashes JVM with a zero exit code (SPARK-16925)") {
// Ensures that if a task which causes the JVM to exit with a zero exit code will cause the
// Spark job to eventually fail.
sc = new SparkContext(clusterUrl, "test")
failAfter(Span(100000, Millis)) {
val thrown = intercept[SparkException] {
sc.parallelize(1 to 1, 1).foreachPartition { _ => System.exit(0) }
}
assert(thrown.getClass === classOf[SparkException])
assert(thrown.getMessage.contains("failed 4 times"))
}
// Check that the cluster is still usable:
sc.parallelize(1 to 10).count()
}

test("caching") {
sc = new SparkContext(clusterUrl, "test")
val data = sc.parallelize(1 to 1000, 10).cache()
Expand Down
10 changes: 4 additions & 6 deletions dev/create-release/release-build.sh
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,7 @@ NEXUS_PROFILE=d63f592e7eac0 # Profile for Spark staging uploads
BASE_DIR=$(pwd)

MVN="build/mvn --force"
PUBLISH_PROFILES="-Pyarn -Phive -Phadoop-2.2"
PUBLISH_PROFILES="-Pyarn -Phive -Phive-thriftserver -Phadoop-2.2"
PUBLISH_PROFILES="$PUBLISH_PROFILES -Pspark-ganglia-lgpl -Pkinesis-asl"

rm -rf spark
Expand Down Expand Up @@ -187,7 +187,7 @@ if [[ "$1" == "package" ]]; then
# We increment the Zinc port each time to avoid OOM's and other craziness if multiple builds
# share the same Zinc server.
make_binary_release "hadoop1" "-Psparkr -Phadoop-1 -Phive -Phive-thriftserver" "3030" &
make_binary_release "hadoop1-scala2.11" "-Psparkr -Phadoop-1 -Phive -Dscala-2.11" "3031" &
make_binary_release "hadoop1-scala2.11" "-Psparkr -Phadoop-1 -Phive -Phive-thriftserver -Dscala-2.11" "3031" &
make_binary_release "cdh4" "-Psparkr -Phadoop-1 -Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" "3032" &
make_binary_release "hadoop2.3" "-Psparkr -Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" "3033" &
make_binary_release "hadoop2.4" "-Psparkr -Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" "3034" &
Expand Down Expand Up @@ -256,8 +256,7 @@ if [[ "$1" == "publish-snapshot" ]]; then
# Generate random point for Zinc
export ZINC_PORT=$(python -S -c "import random; print random.randrange(3030,4030)")

$MVN -DzincPort=$ZINC_PORT --settings $tmp_settings -DskipTests $PUBLISH_PROFILES \
-Phive-thriftserver deploy
$MVN -DzincPort=$ZINC_PORT --settings $tmp_settings -DskipTests $PUBLISH_PROFILES deploy
./dev/change-scala-version.sh 2.11
$MVN -DzincPort=$ZINC_PORT -Dscala-2.11 --settings $tmp_settings \
-DskipTests $PUBLISH_PROFILES clean deploy
Expand Down Expand Up @@ -293,8 +292,7 @@ if [[ "$1" == "publish-release" ]]; then
# Generate random point for Zinc
export ZINC_PORT=$(python -S -c "import random; print random.randrange(3030,4030)")

$MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo -DskipTests $PUBLISH_PROFILES \
-Phive-thriftserver clean install
$MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo -DskipTests $PUBLISH_PROFILES clean install

./dev/change-scala-version.sh 2.11

Expand Down
4 changes: 2 additions & 2 deletions dev/deps/spark-deps-hadoop-1
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ curator-recipes-2.4.0.jar
datanucleus-api-jdo-3.2.6.jar
datanucleus-core-3.2.10.jar
datanucleus-rdbms-3.2.9.jar
derby-10.10.1.1.jar
derby-10.12.1.1.jar
eigenbase-properties-1.1.5.jar
geronimo-annotation_1.0_spec-1.1.1.jar
geronimo-jaspic_1.0_spec-1.0.jar
Expand Down Expand Up @@ -143,7 +143,7 @@ servlet-api-2.5.jar
slf4j-api-1.7.10.jar
slf4j-log4j12-1.7.10.jar
snappy-0.2.jar
snappy-java-1.1.2.1.jar
snappy-java-1.1.2.6.jar
spire-macros_2.10-0.7.4.jar
spire_2.10-0.7.4.jar
stax-api-1.0.1.jar
Expand Down
4 changes: 2 additions & 2 deletions dev/deps/spark-deps-hadoop-2.2
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ curator-recipes-2.4.0.jar
datanucleus-api-jdo-3.2.6.jar
datanucleus-core-3.2.10.jar
datanucleus-rdbms-3.2.9.jar
derby-10.10.1.1.jar
derby-10.12.1.1.jar
eigenbase-properties-1.1.5.jar
geronimo-annotation_1.0_spec-1.1.1.jar
geronimo-jaspic_1.0_spec-1.0.jar
Expand Down Expand Up @@ -173,7 +173,7 @@ servlet-api-2.5.jar
slf4j-api-1.7.10.jar
slf4j-log4j12-1.7.10.jar
snappy-0.2.jar
snappy-java-1.1.2.1.jar
snappy-java-1.1.2.6.jar
spire-macros_2.10-0.7.4.jar
spire_2.10-0.7.4.jar
stax-api-1.0.1.jar
Expand Down
4 changes: 2 additions & 2 deletions dev/deps/spark-deps-hadoop-2.3
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ curator-recipes-2.4.0.jar
datanucleus-api-jdo-3.2.6.jar
datanucleus-core-3.2.10.jar
datanucleus-rdbms-3.2.9.jar
derby-10.10.1.1.jar
derby-10.12.1.1.jar
eigenbase-properties-1.1.5.jar
geronimo-annotation_1.0_spec-1.1.1.jar
geronimo-jaspic_1.0_spec-1.0.jar
Expand Down Expand Up @@ -164,7 +164,7 @@ servlet-api-2.5.jar
slf4j-api-1.7.10.jar
slf4j-log4j12-1.7.10.jar
snappy-0.2.jar
snappy-java-1.1.2.1.jar
snappy-java-1.1.2.6.jar
spire-macros_2.10-0.7.4.jar
spire_2.10-0.7.4.jar
stax-api-1.0-2.jar
Expand Down
Loading