Skip to content

Commit 4ab602a

Browse files
committed
Refactored write ahead stuff from streaming.storage to streaming.util
1 parent b06be2b commit 4ab602a

File tree

7 files changed

+18
-22
lines changed

7 files changed

+18
-22
lines changed

streaming/src/main/scala/org/apache/spark/streaming/storage/FileSegment.scala renamed to streaming/src/main/scala/org/apache/spark/streaming/util/FileSegment.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,6 @@
1414
* See the License for the specific language governing permissions and
1515
* limitations under the License.
1616
*/
17-
package org.apache.spark.streaming.storage
17+
package org.apache.spark.streaming.util
1818

1919
private[streaming] case class FileSegment (path: String, offset: Long, length: Int)

streaming/src/main/scala/org/apache/spark/streaming/storage/HdfsUtils.scala renamed to streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -14,7 +14,7 @@
1414
* See the License for the specific language governing permissions and
1515
* limitations under the License.
1616
*/
17-
package org.apache.spark.streaming.storage
17+
package org.apache.spark.streaming.util
1818

1919
import org.apache.hadoop.conf.Configuration
2020
import org.apache.hadoop.fs.{FSDataInputStream, FSDataOutputStream, Path}
Lines changed: 5 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -14,7 +14,7 @@
1414
* See the License for the specific language governing permissions and
1515
* limitations under the License.
1616
*/
17-
package org.apache.spark.streaming.storage
17+
package org.apache.spark.streaming.util
1818

1919
import java.nio.ByteBuffer
2020

@@ -25,18 +25,17 @@ import org.apache.hadoop.conf.Configuration
2525
import org.apache.hadoop.fs.Path
2626
import org.apache.hadoop.fs.permission.FsPermission
2727
import org.apache.spark.Logging
28-
import org.apache.spark.streaming.storage.WriteAheadLogManager._
29-
import org.apache.spark.streaming.util.{Clock, SystemClock}
3028
import org.apache.spark.util.Utils
29+
import WriteAheadLogManager._
3130

3231
/**
3332
* This class manages write ahead log files.
3433
* - Writes records (bytebuffers) to periodically rotating log files.
3534
* - Recovers the log files and the reads the recovered records upon failures.
3635
* - Cleans up old log files.
3736
*
38-
* Uses [[org.apache.spark.streaming.storage.WriteAheadLogWriter]] to write
39-
* and [[org.apache.spark.streaming.storage.WriteAheadLogReader]] to read.
37+
* Uses [[org.apache.spark.streaming.util.WriteAheadLogWriter]] to write
38+
* and [[org.apache.spark.streaming.util.WriteAheadLogReader]] to read.
4039
*
4140
*@param logDirectory Directory when rotating log files will be created.
4241
* @param hadoopConf Hadoop configuration for reading/writing log files.
@@ -199,7 +198,7 @@ private[streaming] class WriteAheadLogManager(
199198
}
200199
}
201200

202-
private[storage] object WriteAheadLogManager {
201+
private[util] object WriteAheadLogManager {
203202

204203
case class LogInfo(startTime: Long, endTime: Long, path: String)
205204

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -14,7 +14,7 @@
1414
* See the License for the specific language governing permissions and
1515
* limitations under the License.
1616
*/
17-
package org.apache.spark.streaming.storage
17+
package org.apache.spark.streaming.util
1818

1919
import java.io.Closeable
2020
import java.nio.ByteBuffer
@@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration
2323

2424
/**
2525
* A random access reader for reading write ahead log files written using
26-
* [[org.apache.spark.streaming.storage.WriteAheadLogWriter]]. Given the file segment info,
26+
* [[org.apache.spark.streaming.util.WriteAheadLogWriter]]. Given the file segment info,
2727
* this reads the record (bytebuffer) from the log file.
2828
*/
2929
private[streaming] class WriteAheadLogRandomReader(path: String, conf: Configuration)
Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -14,17 +14,17 @@
1414
* See the License for the specific language governing permissions and
1515
* limitations under the License.
1616
*/
17-
package org.apache.spark.streaming.storage
17+
package org.apache.spark.streaming.util
1818

19-
import java.io.{EOFException, Closeable}
19+
import java.io.{Closeable, EOFException}
2020
import java.nio.ByteBuffer
2121

2222
import org.apache.hadoop.conf.Configuration
2323
import org.apache.spark.Logging
2424

2525
/**
2626
* A reader for reading write ahead log files written using
27-
* [[org.apache.spark.streaming.storage.WriteAheadLogWriter]]. This reads
27+
* [[org.apache.spark.streaming.util.WriteAheadLogWriter]]. This reads
2828
* the records (bytebuffers) in the log file sequentially and return them as an
2929
* iterator of bytebuffers.
3030
*/
Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -14,7 +14,7 @@
1414
* See the License for the specific language governing permissions and
1515
* limitations under the License.
1616
*/
17-
package org.apache.spark.streaming.storage
17+
package org.apache.spark.streaming.util
1818

1919
import java.io._
2020
import java.net.URI
@@ -24,7 +24,6 @@ import scala.util.Try
2424

2525
import org.apache.hadoop.conf.Configuration
2626
import org.apache.hadoop.fs.{FSDataOutputStream, FileSystem}
27-
import org.apache.spark.streaming.storage.FileSegment
2827

2928
/**
3029
* A writer for writing byte-buffers to a write ahead log file.

streaming/src/test/scala/org/apache/spark/streaming/storage/WriteAheadLogSuite.scala renamed to streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala

Lines changed: 5 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -14,26 +14,24 @@
1414
* See the License for the specific language governing permissions and
1515
* limitations under the License.
1616
*/
17-
package org.apache.spark.streaming.storage
17+
package org.apache.spark.streaming.util
1818

1919
import java.io.{DataInputStream, File, FileInputStream, RandomAccessFile}
2020
import java.nio.ByteBuffer
2121

2222
import scala.collection.mutable.ArrayBuffer
2323
import scala.concurrent.duration._
2424
import scala.language.implicitConversions
25+
import scala.language.postfixOps
2526
import scala.util.Random
2627

27-
import org.scalatest.{BeforeAndAfter, FunSuite}
28-
import org.scalatest.concurrent.Eventually._
29-
28+
import WriteAheadLogSuite._
3029
import com.google.common.io.Files
3130
import org.apache.commons.io.FileUtils
3231
import org.apache.hadoop.conf.Configuration
33-
34-
import org.apache.spark.streaming.util.ManualClock
3532
import org.apache.spark.util.Utils
36-
import WriteAheadLogSuite._
33+
import org.scalatest.{BeforeAndAfter, FunSuite}
34+
import org.scalatest.concurrent.Eventually._
3735

3836
/**
3937
* This testsuite tests all classes related to write ahead logs.

0 commit comments

Comments
 (0)