aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
blob: a1e98845f6a848b8ab0651aab5e3b0d484d32a02 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.spark.io

import java.io.{InputStream, OutputStream}

import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream}

import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream}
import org.apache.spark.{SparkEnv, SparkConf}


/**
 * CompressionCodec allows the customization of choosing different compression implementations
 * to be used in block storage.
 */
trait CompressionCodec {

  def compressedOutputStream(s: OutputStream): OutputStream

  def compressedInputStream(s: InputStream): InputStream
}


private[spark] object CompressionCodec {
  def createCodec(conf: SparkConf): CompressionCodec = {
    createCodec(conf, conf.get(
      "spark.io.compression.codec", classOf[LZFCompressionCodec].getName))
  }

  def createCodec(conf: SparkConf, codecName: String): CompressionCodec = {
    val ctor = Class.forName(codecName, true, Thread.currentThread.getContextClassLoader)
      .getConstructor(classOf[SparkConf])
    ctor.newInstance(conf).asInstanceOf[CompressionCodec]
  }
}


/**
 * LZF implementation of [[org.apache.spark.io.CompressionCodec]].
 */
class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec {

  override def compressedOutputStream(s: OutputStream): OutputStream = {
    new LZFOutputStream(s).setFinishBlockOnFlush(true)
  }

  override def compressedInputStream(s: InputStream): InputStream = new LZFInputStream(s)
}


/**
 * Snappy implementation of [[org.apache.spark.io.CompressionCodec]].
 * Block size can be configured by spark.io.compression.snappy.block.size.
 */
class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec {

  override def compressedOutputStream(s: OutputStream): OutputStream = {
    val blockSize = conf.get("spark.io.compression.snappy.block.size", "32768").toInt
    new SnappyOutputStream(s, blockSize)
  }

  override def compressedInputStream(s: InputStream): InputStream = new SnappyInputStream(s)
}