You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by qi...@apache.org on 2018/09/03 03:58:29 UTC
carbondata git commit: [CARBONDATA-2903] Fix compiler warning
Repository: carbondata
Updated Branches:
refs/heads/master b588cb655 -> 884ac915b
[CARBONDATA-2903] Fix compiler warning
This closes #2677
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/884ac915
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/884ac915
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/884ac915
Branch: refs/heads/master
Commit: 884ac915bd233a7c0e8b92376c00fb68582e86e3
Parents: b588cb6
Author: Jacky Li <ja...@qq.com>
Authored: Fri Aug 31 16:07:40 2018 +0800
Committer: QiangCai <qi...@qq.com>
Committed: Mon Sep 3 11:57:22 2018 +0800
----------------------------------------------------------------------
.../org/apache/carbondata/mv/datamap/MVHelper.scala | 4 ++--
.../createTable/TestNonTransactionalCarbonTable.scala | 11 +++--------
.../TestSparkCarbonFileFormatWithSparkSession.scala | 3 +--
.../org/apache/spark/sql/profiler/ProfilerSuite.scala | 4 ++--
.../carbondata/spark/rdd/AlterTableAddColumnRDD.scala | 5 +++--
.../carbondata/spark/rdd/AlterTableDropColumnRDD.scala | 5 +++--
.../carbondata/spark/rdd/CarbonDropPartitionRDD.scala | 2 +-
.../carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala | 11 ++++++-----
.../apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala | 6 +-----
.../apache/carbondata/spark/rdd/CarbonMergerRDD.scala | 2 +-
.../org/apache/carbondata/spark/rdd/CarbonRDD.scala | 5 +++--
.../org/apache/carbondata/spark/rdd/CarbonScanRDD.scala | 6 +++---
.../carbondata/spark/rdd/NewCarbonDataLoadRDD.scala | 6 +++---
.../apache/carbondata/spark/rdd/PartitionDropper.scala | 2 ++
.../apache/carbondata/spark/rdd/SparkDataMapJob.scala | 3 ++-
.../apache/carbondata/spark/rdd/StreamHandoffRDD.scala | 3 +--
.../apache/carbondata/spark/util/CarbonScalaUtil.scala | 1 -
.../scala/org/apache/spark/rdd/CarbonMergeFilesRDD.scala | 2 +-
.../org/apache/spark/rdd/DataLoadCoalescedRDD.scala | 4 ++--
.../apache/spark/rdd/DataLoadPartitionCoalescer.scala | 1 +
.../TestCreateTableUsingSparkCarbonFileFormat.scala | 3 +--
.../spark/sql/carbondata/datasource/TestUtil.scala | 4 ++--
.../carbondata/datamap/IndexDataMapRebuildRDD.scala | 4 ++--
.../main/scala/org/apache/spark/sql/CarbonSession.scala | 2 +-
.../spark/sql/execution/CastExpressionOptimization.scala | 2 +-
.../partition/CarbonAlterTableDropPartitionCommand.scala | 2 ++
.../command/preaaggregate/PreAggregateUtil.scala | 2 +-
.../sql/execution/strategy/CarbonDataSourceScan.scala | 4 ++--
.../org/apache/spark/sql/hive/CarbonAnalysisRules.scala | 2 --
.../org/apache/spark/sql/hive/CarbonFileMetastore.scala | 4 +---
.../apache/spark/sql/parser/CarbonSparkSqlParser.scala | 4 ++--
.../spark/carbondata/BadRecordPathLoadOptionTest.scala | 4 +---
.../apache/spark/carbondata/CarbonDataSourceSuite.scala | 2 +-
.../spark/carbondata/TestStreamingTableOperation.scala | 4 ++--
.../carbondata/TestStreamingTableWithRowParser.scala | 2 +-
.../restructure/vectorreader/AddColumnTestCases.scala | 2 +-
.../carbondata/streaming/parser/FieldConverter.scala | 2 +-
37 files changed, 63 insertions(+), 72 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala
----------------------------------------------------------------------
diff --git a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala
index 701f63d..f677826 100644
--- a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala
+++ b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala
@@ -525,8 +525,8 @@ object MVHelper {
aliasMap: Map[AttributeKey, NamedExpression]): Seq[Seq[Any]] = {
val updatedFlagSpec = select.flagSpec.map { f =>
f.map {
- case list: ArrayBuffer[SortOrder] =>
- list.map { s =>
+ case list: ArrayBuffer[_] =>
+ list.map { case s: SortOrder =>
val expressions =
updateOutPutList(
Seq(s.child.asInstanceOf[Attribute]),
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
index c4c20d6..6406a66 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
@@ -176,9 +176,7 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
}
writer.close()
} catch {
- case ex: Exception => throw new RuntimeException(ex)
-
- case _ => None
+ case ex: Throwable => throw new RuntimeException(ex)
}
}
@@ -205,8 +203,7 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
}
writer.close()
} catch {
- case ex: Exception => throw new RuntimeException(ex)
- case _ => None
+ case ex: Throwable => throw new RuntimeException(ex)
}
}
@@ -239,9 +236,7 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
}
writer.close()
} catch {
- case ex: Exception => throw new RuntimeException(ex)
-
- case _ => None
+ case ex: Throwable => throw new RuntimeException(ex)
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestSparkCarbonFileFormatWithSparkSession.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestSparkCarbonFileFormatWithSparkSession.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestSparkCarbonFileFormatWithSparkSession.scala
index 79b64ae..4e2197d 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestSparkCarbonFileFormatWithSparkSession.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestSparkCarbonFileFormatWithSparkSession.scala
@@ -70,8 +70,7 @@ object TestSparkCarbonFileFormatWithSparkSession {
}
writer.close()
} catch {
- case ex: Exception => None
- case _ => None
+ case ex: Throwable => throw new RuntimeException(ex)
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common-test/src/test/scala/org/apache/spark/sql/profiler/ProfilerSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/spark/sql/profiler/ProfilerSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/spark/sql/profiler/ProfilerSuite.scala
index 998e7a9..1ec6aec 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/spark/sql/profiler/ProfilerSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/spark/sql/profiler/ProfilerSuite.scala
@@ -67,7 +67,7 @@ class ProfilerSuite extends QueryTest with BeforeAndAfterAll {
try {
profilerEndPoint.processSQLStart(statementId, messages)
} catch {
- case _ =>
+ case _: Throwable =>
assert(false, "Failed to log StatementSummary")
}
}
@@ -77,7 +77,7 @@ class ProfilerSuite extends QueryTest with BeforeAndAfterAll {
try {
profilerEndPoint.processExecutionEnd(executionId, messages)
} catch {
- case _ =>
+ case _: Throwable =>
assert(false, "Failed to log ExecutionSummary")
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
index f7aa623..d5cce27 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
@@ -48,8 +48,9 @@ class AddColumnPartition(rddId: Int, idx: Int, schema: ColumnSchema) extends Par
/**
* This class is aimed at generating dictionary file for the newly added columns
*/
-class AlterTableAddColumnRDD[K, V](@transient sparkSession: SparkSession,
- @transient newColumns: Seq[ColumnSchema],
+class AlterTableAddColumnRDD[K, V](
+ @transient private val sparkSession: SparkSession,
+ @transient private val newColumns: Seq[ColumnSchema],
identifier: AbsoluteTableIdentifier)
extends CarbonRDD[(Int, SegmentStatus)](sparkSession, Nil) {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
index a0d06b8..ffd20b1 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
@@ -45,8 +45,9 @@ class DropColumnPartition(rddId: Int, idx: Int, schema: ColumnSchema) extends Pa
/**
* This class is aimed at generating dictionary file for the newly added columns
*/
-class AlterTableDropColumnRDD[K, V](@transient ss: SparkSession,
- @transient newColumns: Seq[ColumnSchema],
+class AlterTableDropColumnRDD[K, V](
+ @transient private val ss: SparkSession,
+ @transient private val newColumns: Seq[ColumnSchema],
carbonTableIdentifier: AbsoluteTableIdentifier)
extends CarbonRDD[(Int, SegmentStatus)](ss, Nil) {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDropPartitionRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDropPartitionRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDropPartitionRDD.scala
index e2d1eff..efa92b0 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDropPartitionRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDropPartitionRDD.scala
@@ -43,7 +43,7 @@ case class CarbonDropPartition(rddId: Int, val idx: Int, segment: Segment)
* @param segments segments to be cleaned
*/
class CarbonDropPartitionRDD(
- @transient ss: SparkSession,
+ @transient private val ss: SparkSession,
tablePath: String,
segments: Seq[Segment],
partitions: util.List[PartitionSpec],
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
index 9265c7f..8574e66 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
@@ -174,7 +174,7 @@ case class ColumnDistinctValues(values: Array[String], rowCount: Long) extends S
* @param model a model package load info
*/
class CarbonAllDictionaryCombineRDD(
- @transient sparkSession: SparkSession,
+ @transient private val sparkSession: SparkSession,
prev: RDD[(String, Iterable[String])],
model: DictionaryLoadModel)
extends CarbonRDD[(Int, ColumnDistinctValues)](sparkSession, prev) {
@@ -268,7 +268,7 @@ class StringArrayRow(var values: Array[String]) extends Row {
* @param model a model package load info
*/
class CarbonBlockDistinctValuesCombineRDD(
- @transient ss: SparkSession,
+ @transient private val ss: SparkSession,
prev: RDD[Row],
model: DictionaryLoadModel)
extends CarbonRDD[(Int, ColumnDistinctValues)](ss, prev) {
@@ -327,7 +327,7 @@ class CarbonBlockDistinctValuesCombineRDD(
* @param model a model package load info
*/
class CarbonGlobalDictionaryGenerateRDD(
- @transient sparkSession: SparkSession,
+ @transient private val sparkSession: SparkSession,
prev: RDD[(Int, ColumnDistinctValues)],
model: DictionaryLoadModel)
extends CarbonRDD[(Int, SegmentStatus)](sparkSession, prev) {
@@ -502,9 +502,10 @@ class CarbonColumnDictPatition(id: Int, dimension: CarbonDimension)
* @param dimensions carbon dimenisons having predefined dict
* @param dictFolderPath path of dictionary folder
*/
-class CarbonColumnDictGenerateRDD(carbonLoadModel: CarbonLoadModel,
+class CarbonColumnDictGenerateRDD(
+ carbonLoadModel: CarbonLoadModel,
dictionaryLoadModel: DictionaryLoadModel,
- @transient ss: SparkSession,
+ @transient private val ss: SparkSession,
table: CarbonTableIdentifier,
dimensions: Array[CarbonDimension],
dictFolderPath: String)
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala
index 762b920..8e49fab 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala
@@ -16,12 +16,8 @@
*/
package org.apache.carbondata.spark.rdd
-import java.util
-import java.util.List
-
import scala.collection.JavaConverters._
-import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.mapred.JobConf
import org.apache.hadoop.mapreduce.Job
import org.apache.spark.Partition
@@ -42,7 +38,7 @@ import org.apache.carbondata.spark.MergeResult
* IUD carbon merger RDD
* */
class CarbonIUDMergerRDD[K, V](
- @transient ss: SparkSession,
+ @transient private val ss: SparkSession,
result: MergeResult[K, V],
carbonLoadModel: CarbonLoadModel,
carbonMergerMapping: CarbonMergerMapping,
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index a0425b7..0c30186 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -61,7 +61,7 @@ import org.apache.carbondata.spark.MergeResult
import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, Util}
class CarbonMergerRDD[K, V](
- @transient ss: SparkSession,
+ @transient private val ss: SparkSession,
result: MergeResult[K, V],
carbonLoadModel: CarbonLoadModel,
carbonMergerMapping: CarbonMergerMapping,
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
index 04f20b1..096041f 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
@@ -33,7 +33,8 @@ import org.apache.carbondata.core.util._
/**
* This RDD maintains session level ThreadLocal
*/
-abstract class CarbonRDD[T: ClassTag](@transient ss: SparkSession,
+abstract class CarbonRDD[T: ClassTag](
+ @transient private val ss: SparkSession,
@transient private var deps: Seq[Dependency[_]]) extends RDD[T](ss.sparkContext, deps) {
val carbonSessionInfo: CarbonSessionInfo = {
@@ -86,7 +87,7 @@ abstract class CarbonRDD[T: ClassTag](@transient ss: SparkSession,
* This RDD contains TableInfo object which is serialized and deserialized in driver and executor
*/
abstract class CarbonRDDWithTableInfo[T: ClassTag](
- @transient ss: SparkSession,
+ @transient private val ss: SparkSession,
@transient private var deps: Seq[Dependency[_]],
serializedTableInfo: Array[Byte]) extends CarbonRDD[T](ss, deps) {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index f5d96fc..e88ae81 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -70,12 +70,12 @@ import org.apache.carbondata.streaming.{CarbonStreamInputFormat, CarbonStreamRec
* level filtering in driver side.
*/
class CarbonScanRDD[T: ClassTag](
- @transient spark: SparkSession,
+ @transient private val spark: SparkSession,
val columnProjection: CarbonProjection,
var filterExpression: Expression,
identifier: AbsoluteTableIdentifier,
- @transient serializedTableInfo: Array[Byte],
- @transient tableInfo: TableInfo,
+ @transient private val serializedTableInfo: Array[Byte],
+ @transient private val tableInfo: TableInfo,
inputMetricsStats: InitInputMetrics,
@transient val partitionNames: Seq[PartitionSpec],
val dataTypeConverterClz: Class[_ <: DataTypeConverter] = classOf[SparkDataTypeConverterImpl],
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index 1ada51b..38d6b48 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -160,7 +160,7 @@ class SparkPartitionLoader(model: CarbonLoadModel,
* It loads the data to carbon using @AbstractDataLoadProcessorStep
*/
class NewCarbonDataLoadRDD[K, V](
- @transient ss: SparkSession,
+ @transient private val ss: SparkSession,
result: DataLoadResult[K, V],
carbonLoadModel: CarbonLoadModel,
blocksGroupBy: Array[(String, Array[BlockDetails])])
@@ -307,7 +307,7 @@ class NewCarbonDataLoadRDD[K, V](
* @see org.apache.carbondata.processing.newflow.DataLoadExecutor
*/
class NewDataFrameLoaderRDD[K, V](
- @transient ss: SparkSession,
+ @transient private val ss: SparkSession,
result: DataLoadResult[K, V],
carbonLoadModel: CarbonLoadModel,
prev: DataLoadCoalescedRDD[Row]) extends CarbonRDD[(K, V)](ss, prev) {
@@ -505,7 +505,7 @@ class LazyRddIterator(serializer: SerializerInstance,
* @see org.apache.carbondata.processing.newflow.DataLoadExecutor
*/
class PartitionTableDataLoaderRDD[K, V](
- @transient ss: SparkSession,
+ @transient private val ss: SparkSession,
result: DataLoadResult[K, V],
carbonLoadModel: CarbonLoadModel,
prev: RDD[Row]) extends CarbonRDD[(K, V)](ss, prev) {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
index 3495885..6a4577f 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
@@ -60,6 +60,8 @@ object PartitionDropper {
String.valueOf(oldPartitionIds(partitionIndex + 1))
}
case PartitionType.LIST => "0"
+ case _ => throw new UnsupportedOperationException(
+ s"${partitionInfo.getPartitionType} is not supported")
}
if (!dropWithData) {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/SparkDataMapJob.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/SparkDataMapJob.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/SparkDataMapJob.scala
index b8e73d5..968739b 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/SparkDataMapJob.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/SparkDataMapJob.scala
@@ -57,7 +57,8 @@ class DataMapRDDPartition(rddId: Int, idx: Int, val inputSplit: InputSplit) exte
* @param ss
* @param dataMapFormat
*/
-class DataMapPruneRDD(@transient ss: SparkSession,
+class DataMapPruneRDD(
+ @transient private val ss: SparkSession,
dataMapFormat: DistributableDataMapFormat,
resolverIntf: FilterResolverIntf)
extends CarbonRDD[(ExtendedBlocklet)](ss, Nil) {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
index 39e1875..b63e0dc 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
@@ -98,7 +98,7 @@ class StreamingRawResultIterator(
* execute streaming segment handoff
*/
class StreamHandoffRDD[K, V](
- @transient ss: SparkSession,
+ @transient private val ss: SparkSession,
result: HandoffResult[K, V],
carbonLoadModel: CarbonLoadModel,
handOffSegmentId: String) extends CarbonRDD[(K, V)](ss, Nil) {
@@ -371,7 +371,6 @@ object StreamHandoffRDD {
LOGGER.error("Handoff failed due to failure in table status updation.")
throw new Exception(errorMessage)
}
- done
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
index b779605..2cc2a5b 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
@@ -463,7 +463,6 @@ object CarbonScalaUtil {
}
i += 1
}
- table
} catch {
case e: Exception =>
// ignore it
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common/src/main/scala/org/apache/spark/rdd/CarbonMergeFilesRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/rdd/CarbonMergeFilesRDD.scala b/integration/spark-common/src/main/scala/org/apache/spark/rdd/CarbonMergeFilesRDD.scala
index b5147f0..e29a658 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/rdd/CarbonMergeFilesRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/rdd/CarbonMergeFilesRDD.scala
@@ -41,7 +41,7 @@ case class CarbonMergeFilePartition(rddId: Int, idx: Int, segmentId: String)
* @param segments segments to be merged
*/
class CarbonMergeFilesRDD(
- @transient ss: SparkSession,
+ @transient private val ss: SparkSession,
carbonTable: CarbonTable,
segments: Seq[String],
segmentFileNameToSegmentIdMap: java.util.Map[String, String],
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadCoalescedRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadCoalescedRDD.scala b/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadCoalescedRDD.scala
index 2854c91..2f22b0b 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadCoalescedRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadCoalescedRDD.scala
@@ -28,8 +28,8 @@ import org.apache.carbondata.spark.rdd.CarbonRDD
case class DataLoadPartitionWrap[T: ClassTag](rdd: RDD[T], partition: Partition)
class DataLoadCoalescedRDD[T: ClassTag](
- @transient sparkSession: SparkSession,
- @transient var prev: RDD[T],
+ @transient private val sparkSession: SparkSession,
+ @transient private var prev: RDD[T],
nodeList: Array[String])
extends CarbonRDD[DataLoadPartitionWrap[T]](sparkSession, Nil) {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadPartitionCoalescer.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadPartitionCoalescer.scala b/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadPartitionCoalescer.scala
index 508410a..dfbcf07 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadPartitionCoalescer.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadPartitionCoalescer.scala
@@ -193,6 +193,7 @@ class DataLoadPartitionCoalescer(prev: RDD[_], nodeList: Array[String]) {
hostMapPartitionIds.get(loc) match {
case Some(parts) =>
parts.remove(partitionId)
+ case None =>
}
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestCreateTableUsingSparkCarbonFileFormat.scala
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestCreateTableUsingSparkCarbonFileFormat.scala b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestCreateTableUsingSparkCarbonFileFormat.scala
index 1d9c08a..12b5cbc 100644
--- a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestCreateTableUsingSparkCarbonFileFormat.scala
+++ b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestCreateTableUsingSparkCarbonFileFormat.scala
@@ -80,8 +80,7 @@ class TestCreateTableUsingSparkCarbonFileFormat extends FunSuite with BeforeAndA
}
writer.close()
} catch {
- case ex: Exception => None
- case _ => None
+ case _: Throwable => None
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestUtil.scala b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestUtil.scala
index 1296564..6727ca7 100644
--- a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestUtil.scala
+++ b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestUtil.scala
@@ -48,7 +48,7 @@ object TestUtil {
def checkAnswer(df: DataFrame, expectedAnswer: java.util.List[Row]):Unit = {
checkAnswer(df, expectedAnswer.asScala) match {
case Some(errorMessage) => assert(false, errorMessage)
- case None => null
+ case None =>
}
}
@@ -66,7 +66,7 @@ object TestUtil {
def checkAnswer(df: DataFrame, expectedAnswer: DataFrame): Unit = {
checkAnswer(df, expectedAnswer.collect()) match {
case Some(errorMessage) => assert(false, errorMessage)
- case None => null
+ case None =>
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala b/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala
index 82c64a4..7d6c4e8 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala
@@ -292,9 +292,9 @@ class RawBytesReadSupport(segmentProperties: SegmentProperties, indexColumns: Ar
}
class IndexDataMapRebuildRDD[K, V](
- @transient session: SparkSession,
+ @transient private val session: SparkSession,
result: RefreshResult[K, V],
- @transient tableInfo: TableInfo,
+ @transient private val tableInfo: TableInfo,
dataMapName: String,
indexColumns: Array[CarbonColumn],
segments: Set[Segment])
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
index 5af64ff..96a8162 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
@@ -50,7 +50,7 @@ import org.apache.carbondata.streaming.CarbonStreamingQueryListener
*/
class CarbonSession(@transient val sc: SparkContext,
@transient private val existingSharedState: Option[SharedState],
- @transient useHiveMetaStore: Boolean = true
+ @transient private val useHiveMetaStore: Boolean = true
) extends SparkSession(sc) { self =>
def this(sc: SparkContext) {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CastExpressionOptimization.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CastExpressionOptimization.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CastExpressionOptimization.scala
index 2de3fe6..7e61814 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CastExpressionOptimization.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CastExpressionOptimization.scala
@@ -382,7 +382,7 @@ object CastExpressionOptimization {
numericTimeValue.toString.toDouble
true
} catch {
- case _ => false
+ case _: Throwable => false
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
index 25c0559..cd26fe8 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
@@ -89,6 +89,8 @@ case class CarbonAlterTableDropPartitionCommand(
case PartitionType.RANGE_INTERVAL =>
throwMetadataException(dbName, tableName,
"Dropping range interval partition is unsupported")
+ case _ => throw new UnsupportedOperationException(
+ s"${partitionInfo.getPartitionType} is not supported")
}
partitionInfo.dropPartition(partitionIndex)
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
index ecadf41..f26d1cb 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
@@ -764,7 +764,7 @@ object PreAggregateUtil {
aggExp.isDistinct))
case Sum(_: Expression) =>
Seq(aggExp)
- case Count(MatchCastExpression(exp: Seq[Expression], changeDataType: DataType)) =>
+ case Count(MatchCastExpression(exp: Seq[_], changeDataType: DataType)) =>
Seq(AggregateExpression(Count(Cast(
exp,
changeDataType)),
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
index ccc882d..7605574 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
@@ -30,11 +30,11 @@ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRela
class CarbonDataSourceScan(
override val output: Seq[Attribute],
val rdd: RDD[InternalRow],
- @transient relation: HadoopFsRelation,
+ @transient override val relation: HadoopFsRelation,
val partitioning: Partitioning,
override val metadata: Map[String, String],
identifier: Option[TableIdentifier],
- @transient logicalRelation: LogicalRelation)
+ @transient private val logicalRelation: LogicalRelation)
extends FileSourceScanExec(
relation,
output,
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
index dc8930e..97c37df 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
@@ -272,8 +272,6 @@ case class CarbonPreInsertionCasts(sparkSession: SparkSession) extends Rule[Logi
carbonDSRelation.carbonTable.isHivePartitionTable) {
val newChildOutput = child.output.zipWithIndex.map { columnWithIndex =>
columnWithIndex._1 match {
- case attr: Alias =>
- Alias(attr.child, s"col${ columnWithIndex._2 }")(attr.exprId)
case attr: Attribute =>
Alias(attr, s"col${ columnWithIndex._2 }")(NamedExpression.newExprId)
case attr => attr
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
index dddc72c..70e61bc 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
@@ -620,9 +620,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
}
val tableLocation = catalogTable.storage.locationUri match {
case tableLoc@Some(uri) =>
- if (tableLoc.get.isInstanceOf[String]) {
- FileFactory.getUpdatedFilePath(tableLoc.get.asInstanceOf[String])
- } else if (tableLoc.get.isInstanceOf[URI]) {
+ if (tableLoc.get.isInstanceOf[URI]) {
FileFactory.getUpdatedFilePath(tableLoc.get.asInstanceOf[URI].getPath)
}
case None =>
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
index 39dce3a..b71d035 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
@@ -54,13 +54,13 @@ class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends Ab
case ce: MalformedCarbonCommandException =>
CarbonScalaUtil.cleanParserThreadLocals
throw ce
- case ex =>
+ case ex: Throwable =>
try {
parser.parse(sqlText)
} catch {
case mce: MalformedCarbonCommandException =>
throw mce
- case e =>
+ case e: Throwable =>
CarbonException.analysisException(
s"""== Parse1 ==
|${ex.getMessage}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark2/src/test/scala/org/apache/spark/carbondata/BadRecordPathLoadOptionTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/BadRecordPathLoadOptionTest.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/BadRecordPathLoadOptionTest.scala
index 986365e..dfefa9b 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/BadRecordPathLoadOptionTest.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/BadRecordPathLoadOptionTest.scala
@@ -38,9 +38,7 @@ class BadRecordPathLoadOptionTest extends Spark2QueryTest with BeforeAndAfterAll
var hiveContext: HiveContext = _
override def beforeAll {
- try {
- sql("drop table IF EXISTS salestest")
- }
+ sql("drop table IF EXISTS salestest")
}
test("data load log file and csv file written at the configured location") {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
index 3a67107..7c1265c 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
@@ -173,7 +173,7 @@ class CarbonDataSourceSuite extends Spark2QueryTest with BeforeAndAfterAll {
sql("drop database testdb")
sys.error("drop db should fail as one table exist in db")
} catch {
- case e =>
+ case e: Throwable =>
println(e.getMessage)
}
checkAnswer(sql("select count(*) from testdb.test1"), Seq(Row(2)))
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
index 3073c59..4dde81e 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
@@ -1309,7 +1309,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
sql("ALTER TABLE streaming.stream_table_handoff UNSET TBLPROPERTIES IF EXISTS ('streaming')")
assert(false, "unsupport to unset streaming property")
} catch {
- case _ =>
+ case _: Throwable =>
assert(true)
}
try {
@@ -1327,7 +1327,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
autoHandoff = false
)
} catch {
- case _ =>
+ case _: Throwable =>
assert(false, "should support set table to streaming")
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableWithRowParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableWithRowParser.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableWithRowParser.scala
index d34f7a2..9beee59 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableWithRowParser.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableWithRowParser.scala
@@ -788,7 +788,7 @@ class TestStreamingTableWithRowParser extends QueryTest with BeforeAndAfterAll {
.start()
qry.awaitTermination()
} catch {
- case ex =>
+ case ex: Throwable =>
throw new Exception(ex.getMessage)
} finally {
if (null != qry) {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
index ba42670..363e526 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
@@ -254,7 +254,7 @@ class AddColumnTestCases extends Spark2QueryTest with BeforeAndAfterAll {
assert(true)
}
catch {
- case _ => assert(false)
+ case _: Throwable => assert(false)
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/884ac915/streaming/src/main/scala/org/apache/carbondata/streaming/parser/FieldConverter.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/parser/FieldConverter.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/parser/FieldConverter.scala
index e167d46..cfafd40 100644
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/parser/FieldConverter.scala
+++ b/streaming/src/main/scala/org/apache/carbondata/streaming/parser/FieldConverter.scala
@@ -78,7 +78,7 @@ object FieldConverter {
.append(delimiter)
}
builder.substring(0, builder.length - delimiter.length())
- case m: scala.collection.Map[Any, Any] =>
+ case m: scala.collection.Map[_, _] =>
throw new Exception("Unsupported data type: Map")
case r: org.apache.spark.sql.Row =>
val delimiter = if (level == 1) {