diff options
author | Josh Rosen <joshrosen@databricks.com> | 2015-12-28 14:51:22 -0800 |
---|---|---|
committer | Josh Rosen <joshrosen@databricks.com> | 2015-12-28 14:51:22 -0800 |
commit | fb572c6e4b0645c8084aa013d0c93bb21a79977b (patch) | |
tree | 76289589a51822abf090a0aebb8552f6e973878b /extras | |
parent | a6d385322e7dfaff600465fa5302010a5f122c6b (diff) | |
download | spark-fb572c6e4b0645c8084aa013d0c93bb21a79977b.tar.gz spark-fb572c6e4b0645c8084aa013d0c93bb21a79977b.tar.bz2 spark-fb572c6e4b0645c8084aa013d0c93bb21a79977b.zip |
[SPARK-12525] Fix fatal compiler warnings in Kinesis ASL due to @transient annotations
The Scala 2.11 SBT build currently fails for Spark 1.6.0 and master due to warnings about the `transient` annotation:
```
[error] [warn] /Users/joshrosen/Documents/spark/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala:73: no valid targets for annotation on value sc - it is discarded unused. You may specify targets with meta-annotations, e.g. (transient param)
[error] [warn] transient sc: SparkContext,
```
This fix implemented here is the same as what we did in #8433: remove the `transient` annotations when they are not necessary and replace use `transient private val` in the remaining cases.
Author: Josh Rosen <joshrosen@databricks.com>
Closes #10479 from JoshRosen/fix-sbt-2.11.
Diffstat (limited to 'extras')
2 files changed, 8 insertions, 8 deletions
diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala index 691c1790b2..3996f168e6 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala @@ -70,26 +70,26 @@ class KinesisBackedBlockRDDPartition( */ private[kinesis] class KinesisBackedBlockRDD[T: ClassTag]( - @transient sc: SparkContext, + sc: SparkContext, val regionName: String, val endpointUrl: String, - @transient blockIds: Array[BlockId], + @transient private val _blockIds: Array[BlockId], @transient val arrayOfseqNumberRanges: Array[SequenceNumberRanges], - @transient isBlockIdValid: Array[Boolean] = Array.empty, + @transient private val isBlockIdValid: Array[Boolean] = Array.empty, val retryTimeoutMs: Int = 10000, val messageHandler: Record => T = KinesisUtils.defaultMessageHandler _, val awsCredentialsOption: Option[SerializableAWSCredentials] = None - ) extends BlockRDD[T](sc, blockIds) { + ) extends BlockRDD[T](sc, _blockIds) { - require(blockIds.length == arrayOfseqNumberRanges.length, + require(_blockIds.length == arrayOfseqNumberRanges.length, "Number of blockIds is not equal to the number of sequence number ranges") override def isValid(): Boolean = true override def getPartitions: Array[Partition] = { - Array.tabulate(blockIds.length) { i => + Array.tabulate(_blockIds.length) { i => val isValid = if (isBlockIdValid.length == 0) true else isBlockIdValid(i) - new KinesisBackedBlockRDDPartition(i, blockIds(i), isValid, arrayOfseqNumberRanges(i)) + new KinesisBackedBlockRDDPartition(i, _blockIds(i), isValid, arrayOfseqNumberRanges(i)) } } diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala index 72ab6357a5..3321c7527e 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala @@ -30,7 +30,7 @@ import org.apache.spark.streaming.scheduler.ReceivedBlockInfo import org.apache.spark.streaming.{Duration, StreamingContext, Time} private[kinesis] class KinesisInputDStream[T: ClassTag]( - @transient _ssc: StreamingContext, + _ssc: StreamingContext, streamName: String, endpointUrl: String, regionName: String, |