diff --git a/core/pom.xml b/core/pom.xml
index 1feb00b3a7fb..c5c41b2b5de4 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -352,9 +352,9 @@
-
+
-
+
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
index 819b51e12ad8..4896ec845bbc 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
@@ -19,6 +19,7 @@ package org.apache.spark.scheduler
import java.nio.ByteBuffer
+import scala.language.existentials
import scala.util.control.NonFatal
import org.apache.spark._
diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java
index e5bdad6bda2f..5ce299d05824 100644
--- a/core/src/test/java/org/apache/spark/JavaAPISuite.java
+++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java
@@ -184,6 +184,7 @@ public void sortByKey() {
Assert.assertEquals(new Tuple2(3, 2), sortedPairs.get(2));
}
+ @SuppressWarnings("unchecked")
@Test
public void repartitionAndSortWithinPartitions() {
List> pairs = new ArrayList>();
@@ -491,6 +492,7 @@ public Integer call(Integer a, Integer b) {
Assert.assertEquals(33, sum);
}
+ @SuppressWarnings("unchecked")
@Test
public void aggregateByKey() {
JavaPairRDD pairs = sc.parallelizePairs(
@@ -1556,7 +1558,7 @@ static class Class2 {}
@Test
public void testRegisterKryoClasses() {
SparkConf conf = new SparkConf();
- conf.registerKryoClasses(new Class[]{ Class1.class, Class2.class });
+ conf.registerKryoClasses(new Class>[]{ Class1.class, Class2.class });
Assert.assertEquals(
Class1.class.getName() + "," + Class2.class.getName(),
conf.get("spark.kryo.classesToRegister"));
diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala
index ca226fd4e694..f8bcde12a371 100644
--- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala
@@ -24,14 +24,14 @@ import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.scheduler.{SparkListenerTaskEnd, SparkListener}
import org.scalatest.FunSuite
-import org.scalatest.matchers.ShouldMatchers
+import org.scalatest.Matchers
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{Path, FileSystem}
import scala.collection.mutable.ArrayBuffer
-class InputOutputMetricsSuite extends FunSuite with SharedSparkContext with ShouldMatchers {
+class InputOutputMetricsSuite extends FunSuite with SharedSparkContext with Matchers {
test("input metrics when reading text file with single split") {
val file = new File(getClass.getSimpleName + ".txt")
val pw = new PrintWriter(new FileWriter(file))
diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
index 436eea4f1fdc..d6ec9e129cce 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
@@ -739,7 +739,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F
test("accumulator not calculated for resubmitted result stage") {
//just for register
- val accum = new Accumulator[Int](0, SparkContext.IntAccumulatorParam)
+ val accum = new Accumulator[Int](0, AccumulatorParam.IntAccumulatorParam)
val finalRdd = new MyRDD(sc, 1, Nil)
submit(finalRdd, Array(0))
completeWithAccumulator(accum.id, taskSets(0), Seq((Success, 42)))
diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java
index 064263e02cd1..fbc26167ce66 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java
@@ -49,6 +49,7 @@ public void tearDown() {
public void tfIdf() {
// The tests are to check Java compatibility.
HashingTF tf = new HashingTF();
+ @SuppressWarnings("unchecked")
JavaRDD> documents = sc.parallelize(Lists.newArrayList(
Lists.newArrayList("this is a sentence".split(" ")),
Lists.newArrayList("this is another sentence".split(" ")),
@@ -68,6 +69,7 @@ public void tfIdf() {
public void tfIdfMinimumDocumentFrequency() {
// The tests are to check Java compatibility.
HashingTF tf = new HashingTF();
+ @SuppressWarnings("unchecked")
JavaRDD> documents = sc.parallelize(Lists.newArrayList(
Lists.newArrayList("this is a sentence".split(" ")),
Lists.newArrayList("this is another sentence".split(" ")),
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java
index b751847b464f..f0d079d25b5d 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java
@@ -35,6 +35,7 @@ protected UserDefinedType() { }
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
+ @SuppressWarnings("unchecked")
UserDefinedType that = (UserDefinedType) o;
return this.sqlType().equals(that.sqlType());
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala
index fa37d1f2ae7e..5e68bfebc5ca 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala
@@ -24,8 +24,8 @@ import scala.util.Try
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.mapreduce.Job
-import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter
+import parquet.format.converter.ParquetMetadataConverter
import parquet.hadoop.{ParquetFileReader, Footer, ParquetFileWriter}
import parquet.hadoop.metadata.{ParquetMetadata, FileMetaData}
import parquet.hadoop.util.ContextUtil
@@ -454,7 +454,7 @@ private[parquet] object ParquetTypesConverter extends Logging {
// ... and fallback to "_metadata" if no such file exists (which implies the Parquet file is
// empty, thus normally the "_metadata" file is expected to be fairly small).
.orElse(children.find(_.getPath.getName == ParquetFileWriter.PARQUET_METADATA_FILE))
- .map(ParquetFileReader.readFooter(conf, _))
+ .map(ParquetFileReader.readFooter(conf, _, ParquetMetadataConverter.NO_FILTER))
.getOrElse(
throw new IllegalArgumentException(s"Could not find Parquet metadata at path $path"))
}
diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java
index bc5cd66482ad..2b5812159d07 100644
--- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java
+++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java
@@ -141,6 +141,7 @@ public void constructComplexRow() {
doubleValue, stringValue, timestampValue, null);
// Complex array
+ @SuppressWarnings("unchecked")
List