From 42bcfb2bb2b532dc12e13d3cfc1b4556bbb2c43c Mon Sep 17 00:00:00 2001
From: Matei Zaharia <matei@databricks.com>
Date: Tue, 31 Dec 2013 18:26:23 -0500
Subject: [PATCH] Fix two compile errors introduced in merge

---
 .../src/main/scala/org/apache/spark/streaming/Checkpoint.scala  | 2 +-
 .../org/apache/spark/streaming/scheduler/JobGenerator.scala     | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
index af443279a1..ca0115f90e 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
@@ -64,7 +64,7 @@ class CheckpointWriter(conf: SparkConf, checkpointDir: String, hadoopConf: Confi
   val file = new Path(checkpointDir, "graph")
   val MAX_ATTEMPTS = 3
   val executor = Executors.newFixedThreadPool(1)
-  val compressionCodec = CompressionCodec.createCodec()
+  val compressionCodec = CompressionCodec.createCodec(conf)
   // The file to which we actually write - and then "move" to file
   val writeFile = new Path(file.getParent, file.getName + ".next")
   // The file to which existing checkpoint is backed up (i.e. "moved")
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
index e448211732..ab60a8166e 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
@@ -50,7 +50,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging {
   val clock = {
     val clockClass = ssc.sc.conf.getOrElse(
       "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
-    val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock]
+    Class.forName(clockClass).newInstance().asInstanceOf[Clock]
   }
   val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds,
     longTime => eventProcessorActor ! GenerateJobs(new Time(longTime)))
-- 
GitLab