Repository: spark
Updated Branches:
  refs/heads/master 0fe66744f -> 40ed2af58


http://git-wip-us.apache.org/repos/asf/spark/blob/40ed2af5/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala
index 1a4d41b..392da0b 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala
@@ -20,9 +20,37 @@ package org.apache.spark.sql.sources
 import org.apache.spark.sql.SQLContext
 import org.apache.spark.sql.types.{StringType, StructField, StructType}
 
+
+// please note that the META-INF/services had to be modified for the test 
directory for this to work
+class DDLSourceLoadSuite extends DataSourceTest {
+
+  test("data sources with the same name") {
+    intercept[RuntimeException] {
+      caseInsensitiveContext.read.format("Fluet da Bomb").load()
+    }
+  }
+
+  test("load data source from format alias") {
+    caseInsensitiveContext.read.format("gathering quorum").load().schema ==
+      StructType(Seq(StructField("stringType", StringType, nullable = false)))
+  }
+
+  test("specify full classname with duplicate formats") {
+    
caseInsensitiveContext.read.format("org.apache.spark.sql.sources.FakeSourceOne")
+      .load().schema == StructType(Seq(StructField("stringType", StringType, 
nullable = false)))
+  }
+
+  test("should fail to load ORC without HiveContext") {
+    intercept[ClassNotFoundException] {
+      caseInsensitiveContext.read.format("orc").load()
+    }
+  }
+}
+
+
 class FakeSourceOne extends RelationProvider with DataSourceRegister {
 
-  def format(): String = "Fluet da Bomb"
+  def shortName(): String = "Fluet da Bomb"
 
   override def createRelation(cont: SQLContext, param: Map[String, String]): 
BaseRelation =
     new BaseRelation {
@@ -35,7 +63,7 @@ class FakeSourceOne extends RelationProvider with 
DataSourceRegister {
 
 class FakeSourceTwo extends RelationProvider  with DataSourceRegister {
 
-  def format(): String = "Fluet da Bomb"
+  def shortName(): String = "Fluet da Bomb"
 
   override def createRelation(cont: SQLContext, param: Map[String, String]): 
BaseRelation =
     new BaseRelation {
@@ -48,7 +76,7 @@ class FakeSourceTwo extends RelationProvider  with 
DataSourceRegister {
 
 class FakeSourceThree extends RelationProvider with DataSourceRegister {
 
-  def format(): String = "gathering quorum"
+  def shortName(): String = "gathering quorum"
 
   override def createRelation(cont: SQLContext, param: Map[String, String]): 
BaseRelation =
     new BaseRelation {
@@ -58,28 +86,3 @@ class FakeSourceThree extends RelationProvider with 
DataSourceRegister {
         StructType(Seq(StructField("stringType", StringType, nullable = 
false)))
     }
 }
-// please note that the META-INF/services had to be modified for the test 
directory for this to work
-class DDLSourceLoadSuite extends DataSourceTest {
-
-  test("data sources with the same name") {
-    intercept[RuntimeException] {
-      caseInsensitiveContext.read.format("Fluet da Bomb").load()
-    }
-  }
-
-  test("load data source from format alias") {
-    caseInsensitiveContext.read.format("gathering quorum").load().schema ==
-      StructType(Seq(StructField("stringType", StringType, nullable = false)))
-  }
-
-  test("specify full classname with duplicate formats") {
-    
caseInsensitiveContext.read.format("org.apache.spark.sql.sources.FakeSourceOne")
-      .load().schema == StructType(Seq(StructField("stringType", StringType, 
nullable = false)))
-  }
-
-  test("Loading Orc") {
-    intercept[ClassNotFoundException] {
-      caseInsensitiveContext.read.format("orc").load()
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/40ed2af5/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala
index 3cbf546..27d1cd9 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala
@@ -22,14 +22,39 @@ import 
org.apache.spark.sql.execution.datasources.ResolvedDataSource
 
 class ResolvedDataSourceSuite extends SparkFunSuite {
 
-  test("builtin sources") {
-    assert(ResolvedDataSource.lookupDataSource("jdbc") ===
-      classOf[org.apache.spark.sql.jdbc.DefaultSource])
+  test("jdbc") {
+    assert(
+      ResolvedDataSource.lookupDataSource("jdbc") ===
+      classOf[org.apache.spark.sql.execution.datasources.jdbc.DefaultSource])
+    assert(
+      
ResolvedDataSource.lookupDataSource("org.apache.spark.sql.execution.datasources.jdbc")
 ===
+      classOf[org.apache.spark.sql.execution.datasources.jdbc.DefaultSource])
+    assert(
+      ResolvedDataSource.lookupDataSource("org.apache.spark.sql.jdbc") ===
+        classOf[org.apache.spark.sql.execution.datasources.jdbc.DefaultSource])
+  }
 
-    assert(ResolvedDataSource.lookupDataSource("json") ===
-      classOf[org.apache.spark.sql.json.DefaultSource])
+  test("json") {
+    assert(
+      ResolvedDataSource.lookupDataSource("json") ===
+      classOf[org.apache.spark.sql.execution.datasources.json.DefaultSource])
+    assert(
+      
ResolvedDataSource.lookupDataSource("org.apache.spark.sql.execution.datasources.json")
 ===
+        classOf[org.apache.spark.sql.execution.datasources.json.DefaultSource])
+    assert(
+      ResolvedDataSource.lookupDataSource("org.apache.spark.sql.json") ===
+        classOf[org.apache.spark.sql.execution.datasources.json.DefaultSource])
+  }
 
-    assert(ResolvedDataSource.lookupDataSource("parquet") ===
-      classOf[org.apache.spark.sql.parquet.DefaultSource])
+  test("parquet") {
+    assert(
+      ResolvedDataSource.lookupDataSource("parquet") ===
+      
classOf[org.apache.spark.sql.execution.datasources.parquet.DefaultSource])
+    assert(
+      
ResolvedDataSource.lookupDataSource("org.apache.spark.sql.execution.datasources.parquet")
 ===
+        
classOf[org.apache.spark.sql.execution.datasources.parquet.DefaultSource])
+    assert(
+      ResolvedDataSource.lookupDataSource("org.apache.spark.sql.parquet") ===
+        
classOf[org.apache.spark.sql.execution.datasources.parquet.DefaultSource])
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/40ed2af5/sql/core/src/test/scala/org/apache/spark/sql/ui/SQLListenerSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/ui/SQLListenerSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/ui/SQLListenerSuite.scala
deleted file mode 100644
index 69a561e..0000000
--- a/sql/core/src/test/scala/org/apache/spark/sql/ui/SQLListenerSuite.scala
+++ /dev/null
@@ -1,348 +0,0 @@
-/*
- * 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.sql.ui
-
-import java.util.Properties
-
-import org.apache.spark.{SparkException, SparkContext, SparkConf, 
SparkFunSuite}
-import org.apache.spark.executor.TaskMetrics
-import org.apache.spark.sql.metric.LongSQLMetricValue
-import org.apache.spark.scheduler._
-import org.apache.spark.sql.{DataFrame, SQLContext}
-import org.apache.spark.sql.execution.SQLExecution
-import org.apache.spark.sql.test.TestSQLContext
-
-class SQLListenerSuite extends SparkFunSuite {
-
-  private def createTestDataFrame: DataFrame = {
-    import TestSQLContext.implicits._
-    Seq(
-      (1, 1),
-      (2, 2)
-    ).toDF().filter("_1 > 1")
-  }
-
-  private def createProperties(executionId: Long): Properties = {
-    val properties = new Properties()
-    properties.setProperty(SQLExecution.EXECUTION_ID_KEY, executionId.toString)
-    properties
-  }
-
-  private def createStageInfo(stageId: Int, attemptId: Int): StageInfo = new 
StageInfo(
-    stageId = stageId,
-    attemptId = attemptId,
-    // The following fields are not used in tests
-    name = "",
-    numTasks = 0,
-    rddInfos = Nil,
-    parentIds = Nil,
-    details = ""
-  )
-
-  private def createTaskInfo(taskId: Int, attempt: Int): TaskInfo = new 
TaskInfo(
-    taskId = taskId,
-    attempt = attempt,
-    // The following fields are not used in tests
-    index = 0,
-    launchTime = 0,
-    executorId = "",
-    host = "",
-    taskLocality = null,
-    speculative = false
-  )
-
-  private def createTaskMetrics(accumulatorUpdates: Map[Long, Long]): 
TaskMetrics = {
-    val metrics = new TaskMetrics
-    metrics.setAccumulatorsUpdater(() => accumulatorUpdates.mapValues(new 
LongSQLMetricValue(_)))
-    metrics.updateAccumulators()
-    metrics
-  }
-
-  test("basic") {
-    val listener = new SQLListener(TestSQLContext)
-    val executionId = 0
-    val df = createTestDataFrame
-    val accumulatorIds =
-      
SparkPlanGraph(df.queryExecution.executedPlan).nodes.flatMap(_.metrics.map(_.accumulatorId))
-    // Assume all accumulators are long
-    var accumulatorValue = 0L
-    val accumulatorUpdates = accumulatorIds.map { id =>
-      accumulatorValue += 1L
-      (id, accumulatorValue)
-    }.toMap
-
-    listener.onExecutionStart(
-      executionId,
-      "test",
-      "test",
-      df.queryExecution.toString,
-      SparkPlanGraph(df.queryExecution.executedPlan),
-      System.currentTimeMillis())
-
-    val executionUIData = listener.executionIdToData(0)
-
-    listener.onJobStart(SparkListenerJobStart(
-      jobId = 0,
-      time = System.currentTimeMillis(),
-      stageInfos = Seq(
-        createStageInfo(0, 0),
-        createStageInfo(1, 0)
-      ),
-      createProperties(executionId)))
-    listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(0, 
0)))
-
-    assert(listener.getExecutionMetrics(0).isEmpty)
-
-    listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", 
Seq(
-      // (task id, stage id, stage attempt, metrics)
-      (0L, 0, 0, createTaskMetrics(accumulatorUpdates)),
-      (1L, 0, 0, createTaskMetrics(accumulatorUpdates))
-    )))
-
-    assert(listener.getExecutionMetrics(0) === accumulatorUpdates.mapValues(_ 
* 2))
-
-    listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", 
Seq(
-      // (task id, stage id, stage attempt, metrics)
-      (0L, 0, 0, createTaskMetrics(accumulatorUpdates)),
-      (1L, 0, 0, createTaskMetrics(accumulatorUpdates.mapValues(_ * 2)))
-    )))
-
-    assert(listener.getExecutionMetrics(0) === accumulatorUpdates.mapValues(_ 
* 3))
-
-    // Retrying a stage should reset the metrics
-    listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(0, 
1)))
-
-    listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", 
Seq(
-      // (task id, stage id, stage attempt, metrics)
-      (0L, 0, 1, createTaskMetrics(accumulatorUpdates)),
-      (1L, 0, 1, createTaskMetrics(accumulatorUpdates))
-    )))
-
-    assert(listener.getExecutionMetrics(0) === accumulatorUpdates.mapValues(_ 
* 2))
-
-    // Ignore the task end for the first attempt
-    listener.onTaskEnd(SparkListenerTaskEnd(
-      stageId = 0,
-      stageAttemptId = 0,
-      taskType = "",
-      reason = null,
-      createTaskInfo(0, 0),
-      createTaskMetrics(accumulatorUpdates.mapValues(_ * 100))))
-
-    assert(listener.getExecutionMetrics(0) === accumulatorUpdates.mapValues(_ 
* 2))
-
-    // Finish two tasks
-    listener.onTaskEnd(SparkListenerTaskEnd(
-      stageId = 0,
-      stageAttemptId = 1,
-      taskType = "",
-      reason = null,
-      createTaskInfo(0, 0),
-      createTaskMetrics(accumulatorUpdates.mapValues(_ * 2))))
-    listener.onTaskEnd(SparkListenerTaskEnd(
-      stageId = 0,
-      stageAttemptId = 1,
-      taskType = "",
-      reason = null,
-      createTaskInfo(1, 0),
-      createTaskMetrics(accumulatorUpdates.mapValues(_ * 3))))
-
-    assert(listener.getExecutionMetrics(0) === accumulatorUpdates.mapValues(_ 
* 5))
-
-    // Summit a new stage
-    listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(1, 
0)))
-
-    listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", 
Seq(
-      // (task id, stage id, stage attempt, metrics)
-      (0L, 1, 0, createTaskMetrics(accumulatorUpdates)),
-      (1L, 1, 0, createTaskMetrics(accumulatorUpdates))
-    )))
-
-    assert(listener.getExecutionMetrics(0) === accumulatorUpdates.mapValues(_ 
* 7))
-
-    // Finish two tasks
-    listener.onTaskEnd(SparkListenerTaskEnd(
-      stageId = 1,
-      stageAttemptId = 0,
-      taskType = "",
-      reason = null,
-      createTaskInfo(0, 0),
-      createTaskMetrics(accumulatorUpdates.mapValues(_ * 3))))
-    listener.onTaskEnd(SparkListenerTaskEnd(
-      stageId = 1,
-      stageAttemptId = 0,
-      taskType = "",
-      reason = null,
-      createTaskInfo(1, 0),
-      createTaskMetrics(accumulatorUpdates.mapValues(_ * 3))))
-
-    assert(listener.getExecutionMetrics(0) === accumulatorUpdates.mapValues(_ 
* 11))
-
-    assert(executionUIData.runningJobs === Seq(0))
-    assert(executionUIData.succeededJobs.isEmpty)
-    assert(executionUIData.failedJobs.isEmpty)
-
-    listener.onJobEnd(SparkListenerJobEnd(
-      jobId = 0,
-      time = System.currentTimeMillis(),
-      JobSucceeded
-    ))
-    listener.onExecutionEnd(executionId, System.currentTimeMillis())
-
-    assert(executionUIData.runningJobs.isEmpty)
-    assert(executionUIData.succeededJobs === Seq(0))
-    assert(executionUIData.failedJobs.isEmpty)
-
-    assert(listener.getExecutionMetrics(0) === accumulatorUpdates.mapValues(_ 
* 11))
-  }
-
-  test("onExecutionEnd happens before onJobEnd(JobSucceeded)") {
-    val listener = new SQLListener(TestSQLContext)
-    val executionId = 0
-    val df = createTestDataFrame
-    listener.onExecutionStart(
-      executionId,
-      "test",
-      "test",
-      df.queryExecution.toString,
-      SparkPlanGraph(df.queryExecution.executedPlan),
-      System.currentTimeMillis())
-    listener.onJobStart(SparkListenerJobStart(
-      jobId = 0,
-      time = System.currentTimeMillis(),
-      stageInfos = Nil,
-      createProperties(executionId)))
-    listener.onExecutionEnd(executionId, System.currentTimeMillis())
-    listener.onJobEnd(SparkListenerJobEnd(
-      jobId = 0,
-      time = System.currentTimeMillis(),
-      JobSucceeded
-    ))
-
-    val executionUIData = listener.executionIdToData(0)
-    assert(executionUIData.runningJobs.isEmpty)
-    assert(executionUIData.succeededJobs === Seq(0))
-    assert(executionUIData.failedJobs.isEmpty)
-  }
-
-  test("onExecutionEnd happens before multiple onJobEnd(JobSucceeded)s") {
-    val listener = new SQLListener(TestSQLContext)
-    val executionId = 0
-    val df = createTestDataFrame
-    listener.onExecutionStart(
-      executionId,
-      "test",
-      "test",
-      df.queryExecution.toString,
-      SparkPlanGraph(df.queryExecution.executedPlan),
-      System.currentTimeMillis())
-    listener.onJobStart(SparkListenerJobStart(
-      jobId = 0,
-      time = System.currentTimeMillis(),
-      stageInfos = Nil,
-      createProperties(executionId)))
-    listener.onJobEnd(SparkListenerJobEnd(
-        jobId = 0,
-        time = System.currentTimeMillis(),
-        JobSucceeded
-    ))
-
-    listener.onJobStart(SparkListenerJobStart(
-      jobId = 1,
-      time = System.currentTimeMillis(),
-      stageInfos = Nil,
-      createProperties(executionId)))
-    listener.onExecutionEnd(executionId, System.currentTimeMillis())
-    listener.onJobEnd(SparkListenerJobEnd(
-      jobId = 1,
-      time = System.currentTimeMillis(),
-      JobSucceeded
-    ))
-
-    val executionUIData = listener.executionIdToData(0)
-    assert(executionUIData.runningJobs.isEmpty)
-    assert(executionUIData.succeededJobs.sorted === Seq(0, 1))
-    assert(executionUIData.failedJobs.isEmpty)
-  }
-
-  test("onExecutionEnd happens before onJobEnd(JobFailed)") {
-    val listener = new SQLListener(TestSQLContext)
-    val executionId = 0
-    val df = createTestDataFrame
-    listener.onExecutionStart(
-      executionId,
-      "test",
-      "test",
-      df.queryExecution.toString,
-      SparkPlanGraph(df.queryExecution.executedPlan),
-      System.currentTimeMillis())
-    listener.onJobStart(SparkListenerJobStart(
-      jobId = 0,
-      time = System.currentTimeMillis(),
-      stageInfos = Seq.empty,
-      createProperties(executionId)))
-    listener.onExecutionEnd(executionId, System.currentTimeMillis())
-    listener.onJobEnd(SparkListenerJobEnd(
-      jobId = 0,
-      time = System.currentTimeMillis(),
-      JobFailed(new RuntimeException("Oops"))
-    ))
-
-    val executionUIData = listener.executionIdToData(0)
-    assert(executionUIData.runningJobs.isEmpty)
-    assert(executionUIData.succeededJobs.isEmpty)
-    assert(executionUIData.failedJobs === Seq(0))
-  }
-
-  ignore("no memory leak") {
-    val conf = new SparkConf()
-      .setMaster("local")
-      .setAppName("test")
-      .set("spark.task.maxFailures", "1") // Don't retry the tasks to run this 
test quickly
-      .set("spark.sql.ui.retainedExecutions", "50") // Set it to 50 to run 
this test quickly
-    val sc = new SparkContext(conf)
-    try {
-      val sqlContext = new SQLContext(sc)
-      import sqlContext.implicits._
-      // Run 100 successful executions and 100 failed executions.
-      // Each execution only has one job and one stage.
-      for (i <- 0 until 100) {
-        val df = Seq(
-          (1, 1),
-          (2, 2)
-        ).toDF()
-        df.collect()
-        try {
-          df.foreach(_ => throw new RuntimeException("Oops"))
-        } catch {
-          case e: SparkException => // This is expected for a failed job
-        }
-      }
-      sc.listenerBus.waitUntilEmpty(10000)
-      assert(sqlContext.listener.getCompletedExecutions.size <= 50)
-      assert(sqlContext.listener.getFailedExecutions.size <= 50)
-      // 50 for successful executions and 50 for failed executions
-      assert(sqlContext.listener.executionIdToData.size <= 100)
-      assert(sqlContext.listener.jobIdToExecutionId.size <= 100)
-      assert(sqlContext.listener.stageIdToStageMetrics.size <= 100)
-    } finally {
-      sc.stop()
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/40ed2af5/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index 7198a32..ac9aaed 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -41,7 +41,7 @@ import org.apache.spark.sql.catalyst.{InternalRow, SqlParser, 
TableIdentifier}
 import org.apache.spark.sql.execution.datasources
 import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, 
LogicalRelation, Partition => ParquetPartition, PartitionSpec, 
ResolvedDataSource}
 import org.apache.spark.sql.hive.client._
-import org.apache.spark.sql.parquet.ParquetRelation
+import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation
 import org.apache.spark.sql.sources._
 import org.apache.spark.sql.types._
 import org.apache.spark.sql.{AnalysisException, SQLContext, SaveMode}

http://git-wip-us.apache.org/repos/asf/spark/blob/40ed2af5/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala
index 0c344c6..9f4f8b5 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala
@@ -32,7 +32,6 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
 import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext}
 
 import org.apache.spark.Logging
-import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.mapred.SparkHadoopMapRedUtil
 import org.apache.spark.rdd.{HadoopRDD, RDD}
 import org.apache.spark.sql.catalyst.InternalRow
@@ -49,9 +48,9 @@ import scala.collection.JavaConversions._
 
 private[sql] class DefaultSource extends HadoopFsRelationProvider with 
DataSourceRegister {
 
-  def format(): String = "orc"
+  override def shortName(): String = "orc"
 
-  def createRelation(
+  override def createRelation(
       sqlContext: SQLContext,
       paths: Array[String],
       dataSchema: Option[StructType],
@@ -144,7 +143,6 @@ private[orc] class OrcOutputWriter(
   }
 }
 
-@DeveloperApi
 private[sql] class OrcRelation(
     override val paths: Array[String],
     maybeDataSchema: Option[StructType],

http://git-wip-us.apache.org/repos/asf/spark/blob/40ed2af5/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
index a45c2d9..1fa005d 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.hive
 
 import org.apache.spark.sql.hive.test.TestHive
-import org.apache.spark.sql.parquet.ParquetTest
+import org.apache.spark.sql.execution.datasources.parquet.ParquetTest
 import org.apache.spark.sql.{QueryTest, Row}
 
 case class Cases(lower: String, UPPER: String)

http://git-wip-us.apache.org/repos/asf/spark/blob/40ed2af5/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
index b73d666..7f36a48 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
@@ -32,7 +32,7 @@ import org.apache.spark.sql.hive.client.{HiveTable, 
ManagedTable}
 import org.apache.spark.sql.hive.test.TestHive
 import org.apache.spark.sql.hive.test.TestHive._
 import org.apache.spark.sql.hive.test.TestHive.implicits._
-import org.apache.spark.sql.parquet.ParquetRelation
+import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation
 import org.apache.spark.sql.test.SQLTestUtils
 import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils

http://git-wip-us.apache.org/repos/asf/spark/blob/40ed2af5/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala
index f00d375..80eb9f1 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.hive
 import org.apache.hadoop.hive.conf.HiveConf
 
 import org.apache.spark.sql.hive.test.TestHive
-import org.apache.spark.sql.parquet.ParquetCompatibilityTest
+import 
org.apache.spark.sql.execution.datasources.parquet.ParquetCompatibilityTest
 import org.apache.spark.sql.{Row, SQLConf, SQLContext}
 
 class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest {

http://git-wip-us.apache.org/repos/asf/spark/blob/40ed2af5/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
index 2fa7ae3..79a136a 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
@@ -30,7 +30,7 @@ import org.apache.spark.sql.hive.test.TestHive
 import org.apache.spark.sql.hive.test.TestHive._
 import org.apache.spark.sql.hive.test.TestHive.implicits._
 import org.apache.spark.sql.hive.{HiveContext, HiveQLDialect, 
MetastoreRelation}
-import org.apache.spark.sql.parquet.ParquetRelation
+import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation
 import org.apache.spark.sql.test.SQLTestUtils
 import org.apache.spark.sql.types._
 import org.apache.spark.unsafe.types.CalendarInterval

http://git-wip-us.apache.org/repos/asf/spark/blob/40ed2af5/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
index c4bc600..50f0243 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
@@ -28,7 +28,7 @@ import org.apache.spark.sql.hive.execution.HiveTableScan
 import org.apache.spark.sql.hive.test.TestHive
 import org.apache.spark.sql.hive.test.TestHive._
 import org.apache.spark.sql.hive.test.TestHive.implicits._
-import org.apache.spark.sql.parquet.ParquetRelation
+import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation
 import org.apache.spark.sql.test.SQLTestUtils
 import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils

http://git-wip-us.apache.org/repos/asf/spark/blob/40ed2af5/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala
index d280543..cb4cedd 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala
@@ -23,12 +23,12 @@ import com.google.common.io.Files
 import org.apache.hadoop.fs.Path
 
 import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.sql.{AnalysisException, SaveMode, parquet}
+import org.apache.spark.sql.{AnalysisException, SaveMode}
 import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
 
 
 class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest {
-  override val dataSourceName: String = 
classOf[parquet.DefaultSource].getCanonicalName
+  override val dataSourceName: String = "parquet"
 
   import sqlContext._
   import sqlContext.implicits._

http://git-wip-us.apache.org/repos/asf/spark/blob/40ed2af5/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala
index 1813cc3..48c37a1 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala
@@ -53,7 +53,7 @@ class SimpleTextHadoopFsRelationSuite extends 
HadoopFsRelationTest {
 
 class JsonHadoopFsRelationSuite extends HadoopFsRelationTest {
   override val dataSourceName: String =
-    classOf[org.apache.spark.sql.json.DefaultSource].getCanonicalName
+    
classOf[org.apache.spark.sql.execution.datasources.json.DefaultSource].getCanonicalName
 
   import sqlContext._
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to