Skip to content

Commit

Permalink
use a internal config to indicate test is ongoing (#297)
Browse files Browse the repository at this point in the history
* use a internal config to indicate test is ongoing

* minor fix
  • Loading branch information
Liu, Linhong authored Jul 12, 2017
1 parent 2af85e6 commit 2a8d8db
Show file tree
Hide file tree
Showing 11 changed files with 30 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -188,12 +188,14 @@ private[oap] class OapDataReader(
*/
val indexFileSize = indexPath.getFileSystem(conf).getContentSummary(indexPath).getLength
val dataFileSize = path.getFileSystem(conf).getContentSummary(path).getLength
val isTesting = conf.getBoolean(SQLConf.OAP_IS_TESTING.key,
SQLConf.OAP_IS_TESTING.defaultValue.get)
val iter =
if (indexFileSize > FiberCacheManager.getMaximumFiberSizeInBytes(conf)) {
logWarning(s"Index File size $indexFileSize B is too large and couldn't be cached." +
s"Please increase ${SQLConf.OAP_FIBERCACHE_SIZE.key} for better performance")
fileScanner.iterator(conf, requiredIds)
} else if (indexFileSize > dataFileSize * 0.7) {
} else if (indexFileSize > dataFileSize * 0.7 && !isTesting) {
logWarning(s"Index File size $indexFileSize B is too large comparing " +
s"to Data File Size $dataFileSize. Using Data File Scan instead.")
fileScanner.iterator(conf, requiredIds)
Expand Down
7 changes: 7 additions & 0 deletions src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
Original file line number Diff line number Diff line change
Expand Up @@ -641,6 +641,13 @@ object SQLConf {
.intConf
.createWithDefault(1024 * 1024)

val OAP_IS_TESTING =
SQLConfigBuilder("spark.sql.oap.testing")
.internal()
.doc("To indicate if the test is ongoing")
.booleanConf
.createWithDefault(false)


object Deprecated {
val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import org.scalatest.BeforeAndAfter

import org.apache.spark.sql.{Row, SaveMode}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.types.{IntegerType, StringType, StructType}
import org.apache.spark.util.Utils
Expand All @@ -38,6 +39,7 @@ class DataSourceMetaSuite extends SharedSQLContext with BeforeAndAfter {

override def beforeAll(): Unit = {
super.beforeAll()
sqlContext.conf.setConf(SQLConf.OAP_IS_TESTING, true)
tmpDir = Utils.createTempDir()
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ class FilterSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEac
import testImplicits._

override def beforeEach(): Unit = {
sqlContext.conf.setConf(SQLConf.OAP_IS_TESTING, true)
val path = Utils.createTempDir().getAbsolutePath

sql(s"""CREATE TEMPORARY VIEW oap_test (a INT, b STRING)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path
import org.scalatest.BeforeAndAfterEach

import org.apache.spark.sql.{QueryTest, Row}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.util.Utils

Expand All @@ -30,6 +31,7 @@ class OapDDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEac
import testImplicits._

override def beforeEach(): Unit = {
sqlContext.conf.setConf(SQLConf.OAP_IS_TESTING, true)
val path1 = Utils.createTempDir().getAbsolutePath
val path2 = Utils.createTempDir().getAbsolutePath

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ class OapIndexQuerySuite extends QueryTest with SharedSQLContext with BeforeAndA
import testImplicits._

override def beforeEach(): Unit = {
sqlContext.conf.setConf(SQLConf.OAP_IS_TESTING, true)
val path1 = Utils.createTempDir().getAbsolutePath

sql(s"""CREATE TEMPORARY VIEW oap_test_1 (a INT, b STRING)
Expand All @@ -41,8 +42,8 @@ class OapIndexQuerySuite extends QueryTest with SharedSQLContext with BeforeAndA
}

test("index integrity") {
val data: Seq[(Int, String)] = scala.util.Random.shuffle(1 to 300).map{
i => (i, s"this is test $i") }.toSeq
val data: Seq[(Int, String)] =
scala.util.Random.shuffle(1 to 300).map{ i => (i, s"this is test $i") }.toSeq
data.toDF("key", "value").createOrReplaceTempView("t")
sql("insert overwrite table oap_test_1 select * from t")
sql("create oindex index1 on oap_test_1 (a) using bitmap")
Expand All @@ -56,7 +57,7 @@ class OapIndexQuerySuite extends QueryTest with SharedSQLContext with BeforeAndA
}

test("Large Bloom Bit Size Cause JVM crash. Issue #278") {
sqlContext.conf.setConfString(SQLConf.OAP_BLOOMFILTER_MAXBITS.key, s"${1 << 30}")
sqlContext.conf.setConf(SQLConf.OAP_BLOOMFILTER_MAXBITS, 1 << 30)

val data: Seq[(Int, String)] = (1 to 300).map { i => (i, s"this is test $i") }
data.toDF("key", "value").createOrReplaceTempView("t")
Expand All @@ -66,6 +67,8 @@ class OapIndexQuerySuite extends QueryTest with SharedSQLContext with BeforeAndA
checkAnswer(sql("SELECT * FROM oap_test_1 WHERE a = 1"), Row(1, "this is test 1") :: Nil)

sql("drop oindex index1 on oap_test_1")
sqlContext.conf.setConf(SQLConf.OAP_BLOOMFILTER_MAXBITS,
SQLConf.OAP_BLOOMFILTER_MAXBITS.defaultValue.get)
}

test("index row boundary") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ class OapSuite extends QueryTest with SharedSQLContext with BeforeAndAfter {

override def beforeAll(): Unit = {
super.beforeAll()
sqlContext.conf.setConf(SQLConf.OAP_IS_TESTING, true)
path = Utils.createTempDir()
path.delete()
parquetPath = Utils.createTempDir()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.oap.index
import org.scalatest.BeforeAndAfterEach

import org.apache.spark.sql.{QueryTest, Row}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.util.Utils

Expand All @@ -31,6 +32,7 @@ class BitMapIndexSuite extends QueryTest with SharedSQLContext with BeforeAndAft
import testImplicits._

override def beforeEach(): Unit = {
sqlContext.conf.setConf(SQLConf.OAP_IS_TESTING, true)
val path = Utils.createTempDir().getAbsolutePath
sql(s"""CREATE TEMPORARY VIEW oap_test (a INT, b STRING)
| USING oap
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,9 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.scalatest.BeforeAndAfterEach

import org.apache.spark.sql.SaveMode
import org.apache.spark.sql.execution.datasources.oap.{DataSourceMeta, OapFileFormat}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.SaveMode
import org.apache.spark.sql.sources._
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.util.Utils
Expand All @@ -37,6 +38,7 @@ class IndexSelectionSuite extends SharedSQLContext with BeforeAndAfterEach{
private var path: Path = null

override def beforeEach(): Unit = {
sqlContext.conf.setConf(SQLConf.OAP_IS_TESTING, true)
val data = (1 to 300).map(i => (i, i + 100, i + 200, i + 300, s"this is row $i"))
val df = sparkContext.parallelize(data, 3).toDF("a", "b", "c", "d", "e")
df.write.format("oap").mode(SaveMode.Overwrite).save(tempDir.getAbsolutePath)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import org.scalatest.BeforeAndAfterEach

import org.apache.spark.sql.{QueryTest, Row}
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.util.Utils

Expand All @@ -31,6 +32,7 @@ class StatisticsManagerSuite extends QueryTest with SharedSQLContext with Before
import testImplicits._

override def beforeEach(): Unit = {
sqlContext.conf.setConf(SQLConf.OAP_IS_TESTING, true)
val path = Utils.createTempDir().getAbsolutePath

sql(s"""CREATE TEMPORARY VIEW oap_test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ class OapQuerySuite extends HiveComparisonTest with BeforeAndAfter {

override def beforeAll() {
super.beforeAll()
sparkContext.conf.set(SQLConf.OAP_IS_TESTING.key, SQLConf.OAP_IS_TESTING.defaultValueString)
TestHive.setCacheTables(true)
// Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*)
TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles"))
Expand Down

0 comments on commit 2a8d8db

Please sign in to comment.