Skip to content

Commit

Permalink
update
Browse files Browse the repository at this point in the history
  • Loading branch information
云峤 committed May 3, 2015
1 parent 7394fd5 commit 03ef434
Show file tree
Hide file tree
Showing 2 changed files with 29 additions and 33 deletions.
57 changes: 24 additions & 33 deletions sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,31 +20,31 @@ package org.apache.spark.sql
import java.io.CharArrayWriter
import java.sql.DriverManager

import scala.collection.JavaConversions._
import scala.language.implicitConversions
import scala.reflect.ClassTag
import scala.reflect.runtime.universe.TypeTag
import scala.util.control.NonFatal

import com.fasterxml.jackson.core.JsonFactory

import org.apache.commons.lang3.StringUtils
import org.apache.spark.annotation.{DeveloperApi, Experimental}
import org.apache.spark.api.java.JavaRDD
import org.apache.spark.api.python.SerDeUtil
import org.apache.spark.rdd.RDD
import org.apache.spark.storage.StorageLevel
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, SqlParser}
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation, ResolvedStar}
import org.apache.spark.sql.catalyst.analysis.{ResolvedStar, UnresolvedAttribute, UnresolvedRelation}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.{JoinType, Inner}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.{Inner, JoinType}
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, SqlParser}
import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, LogicalRDD}
import org.apache.spark.sql.jdbc.JDBCWriteDetails
import org.apache.spark.sql.json.JsonRDD
import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, ResolvedDataSource}
import org.apache.spark.sql.types._
import org.apache.spark.sql.sources.{ResolvedDataSource, CreateTableUsingAsSelect}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.Utils

import scala.collection.JavaConversions._
import scala.language.implicitConversions
import scala.reflect.ClassTag
import scala.reflect.runtime.universe.TypeTag
import scala.util.control.NonFatal


private[sql] object DataFrame {
def apply(sqlContext: SQLContext, logicalPlan: LogicalPlan): DataFrame = {
Expand Down Expand Up @@ -196,32 +196,23 @@ class DataFrame private[sql](
}

// Create SeparateLine
val sep:String = {
"+" + colWidths.map {
size =>
val columnSep = new Array[Char](size)
for (i <- 0 until size)
columnSep(i) = '-'
String.valueOf(columnSep)
}.mkString("+") + "+\n"
}
val sep: String = colWidths.map("-" * _).addString(sb, "+", "+", "+\n").toString()

sb.append(sep)
// column names
rows.head.zipWithIndex.map { case (cell, i) =>
StringUtils.leftPad(cell.toString, colWidths(i))
}.addString(sb, "|", "|", "|\n")

// append column names
sb.append("|").append(rows.head.zipWithIndex.map { case (cell, i) =>
String.format(s"%${colWidths(i)}s", cell)
}.mkString("|")).append("|\n")
sb.append(sep)

// append data
sb.append(rows.tail.map { row =>
"|" + row.zipWithIndex.map { case (cell, i) =>
String.format(s"%${colWidths(i)}s", cell)
}.mkString("|") + "|"
}.mkString("\n")).append("\n")
// data
rows.tail.map {
_.zipWithIndex.map { case (cell, i) =>
StringUtils.leftPad(cell.toString, colWidths(i))
}.addString(sb, "|", "|", "|\n")
}

sb.append(sep)

sb.toString()
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -598,6 +598,11 @@ class DataFrameSuite extends QueryTest {
testData.select($"*").show(1000)
}

test("SPARK-7319 showString") {
assert(testData.select($"*").showString(1).split("\n") === Seq("+---+-----+",
"|key|value|", "+---+-----+", "| 1| 1|", "+---+-----+"))
}

test("createDataFrame(RDD[Row], StructType) should convert UDTs (SPARK-6672)") {
val rowRDD = TestSQLContext.sparkContext.parallelize(Seq(Row(new ExamplePoint(1.0, 2.0))))
val schema = StructType(Array(StructField("point", new ExamplePointUDT(), false)))
Expand Down

0 comments on commit 03ef434

Please sign in to comment.