case class StructType(fields: Array[StructField]) extends DataType with Seq[StructField] with Product with Serializable
A StructType object can be constructed by
StructType(fields: Seq[StructField])
For a StructType object, one or multiple StructFields can be extracted by names.
If multiple StructFields are extracted, a StructType object will be returned.
If a provided name does not have a matching field, it will be ignored. For the case
of extracting a single StructField, a null
will be returned.
Scala Example:
import org.apache.spark.sql._ import org.apache.spark.sql.types._ val struct = StructType( StructField("a", IntegerType, true) :: StructField("b", LongType, false) :: StructField("c", BooleanType, false) :: Nil) // Extract a single StructField. val singleField = struct("b") // singleField: StructField = StructField(b,LongType,false) // If this struct does not have a field called "d", it throws an exception. struct("d") // java.lang.IllegalArgumentException: d does not exist. // ... // Extract multiple StructFields. Field names are provided in a set. // A StructType object will be returned. val twoFields = struct(Set("b", "c")) // twoFields: StructType = // StructType(StructField(b,LongType,false), StructField(c,BooleanType,false)) // Any names without matching fields will throw an exception. // For the case shown below, an exception is thrown due to "d". struct(Set("b", "c", "d")) // java.lang.IllegalArgumentException: d does not exist. // ...
A org.apache.spark.sql.Row object is used as a value of the StructType.
Scala Example:
import org.apache.spark.sql._ import org.apache.spark.sql.types._ val innerStruct = StructType( StructField("f1", IntegerType, true) :: StructField("f2", LongType, false) :: StructField("f3", BooleanType, false) :: Nil) val struct = StructType( StructField("a", innerStruct, true) :: Nil) // Create a Row with the schema defined by struct val row = Row(Row(1, 2, true))
- Annotations
- @Stable()
- Source
- StructType.scala
- Since
1.3.0
- Alphabetic
- By Inheritance
- StructType
- Serializable
- Serializable
- Product
- Seq
- SeqLike
- GenSeq
- GenSeqLike
- Iterable
- IterableLike
- Equals
- GenIterable
- GenIterableLike
- Traversable
- GenTraversable
- GenericTraversableTemplate
- TraversableLike
- GenTraversableLike
- Parallelizable
- TraversableOnce
- GenTraversableOnce
- FilterMonadic
- HasNewBuilder
- PartialFunction
- Function1
- DataType
- AbstractDataType
- AnyRef
- Any
- Hide All
- Show All
- Public
- All
Instance Constructors
-
new
StructType()
No-arg constructor for kryo.
- new StructType(fields: Array[StructField])
Type Members
-
type
Self = Seq[StructField]
- Attributes
- protected[this]
- Definition Classes
- TraversableLike
-
class
WithFilter extends FilterMonadic[A, Repr]
- Definition Classes
- TraversableLike
Value Members
-
final
def
!=(arg0: Any): Boolean
- Definition Classes
- AnyRef → Any
-
final
def
##(): Int
- Definition Classes
- AnyRef → Any
-
def
++[B >: StructField, That](that: GenTraversableOnce[B])(implicit bf: CanBuildFrom[Seq[StructField], B, That]): That
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
++:[B >: StructField, That](that: Traversable[B])(implicit bf: CanBuildFrom[Seq[StructField], B, That]): That
- Definition Classes
- TraversableLike
-
def
++:[B >: StructField, That](that: TraversableOnce[B])(implicit bf: CanBuildFrom[Seq[StructField], B, That]): That
- Definition Classes
- TraversableLike
-
def
+:[B >: StructField, That](elem: B)(implicit bf: CanBuildFrom[Seq[StructField], B, That]): That
- Definition Classes
- SeqLike → GenSeqLike
-
def
:+[B >: StructField, That](elem: B)(implicit bf: CanBuildFrom[Seq[StructField], B, That]): That
- Definition Classes
- SeqLike → GenSeqLike
-
final
def
==(arg0: Any): Boolean
- Definition Classes
- AnyRef → Any
-
def
add(name: String, dataType: String, nullable: Boolean, comment: String): StructType
Creates a new StructType by adding a new field and specifying metadata where the dataType is specified as a String.
Creates a new StructType by adding a new field and specifying metadata where the dataType is specified as a String.
val struct = (new StructType) .add("a", "int", true, "comment1") .add("b", "long", false, "comment2") .add("c", "string", true, "comment3")
-
def
add(name: String, dataType: String, nullable: Boolean, metadata: Metadata): StructType
Creates a new StructType by adding a new field and specifying metadata where the dataType is specified as a String.
Creates a new StructType by adding a new field and specifying metadata where the dataType is specified as a String.
val struct = (new StructType) .add("a", "int", true, Metadata.empty) .add("b", "long", false, Metadata.empty) .add("c", "string", true, Metadata.empty)
-
def
add(name: String, dataType: String, nullable: Boolean): StructType
Creates a new StructType by adding a new field with no metadata where the dataType is specified as a String.
Creates a new StructType by adding a new field with no metadata where the dataType is specified as a String.
val struct = (new StructType) .add("a", "int", true) .add("b", "long", false) .add("c", "string", true)
-
def
add(name: String, dataType: String): StructType
Creates a new StructType by adding a new nullable field with no metadata where the dataType is specified as a String.
Creates a new StructType by adding a new nullable field with no metadata where the dataType is specified as a String.
val struct = (new StructType) .add("a", "int") .add("b", "long") .add("c", "string")
-
def
add(name: String, dataType: DataType, nullable: Boolean, comment: String): StructType
Creates a new StructType by adding a new field and specifying metadata.
Creates a new StructType by adding a new field and specifying metadata.
val struct = (new StructType) .add("a", IntegerType, true, "comment1") .add("b", LongType, false, "comment2") .add("c", StringType, true, "comment3")
-
def
add(name: String, dataType: DataType, nullable: Boolean, metadata: Metadata): StructType
Creates a new StructType by adding a new field and specifying metadata.
Creates a new StructType by adding a new field and specifying metadata.
val struct = (new StructType) .add("a", IntegerType, true, Metadata.empty) .add("b", LongType, false, Metadata.empty) .add("c", StringType, true, Metadata.empty)
-
def
add(name: String, dataType: DataType, nullable: Boolean): StructType
Creates a new StructType by adding a new field with no metadata.
Creates a new StructType by adding a new field with no metadata.
val struct = (new StructType) .add("a", IntegerType, true) .add("b", LongType, false) .add("c", StringType, true)
-
def
add(name: String, dataType: DataType): StructType
Creates a new StructType by adding a new nullable field with no metadata.
Creates a new StructType by adding a new nullable field with no metadata.
val struct = (new StructType) .add("a", IntegerType) .add("b", LongType) .add("c", StringType)
-
def
add(field: StructField): StructType
Creates a new StructType by adding a new field.
Creates a new StructType by adding a new field.
val struct = (new StructType) .add(StructField("a", IntegerType, true)) .add(StructField("b", LongType, false)) .add(StructField("c", StringType, true))
-
def
addString(b: StringBuilder): StringBuilder
- Definition Classes
- TraversableOnce
-
def
addString(b: StringBuilder, sep: String): StringBuilder
- Definition Classes
- TraversableOnce
-
def
addString(b: StringBuilder, start: String, sep: String, end: String): StringBuilder
- Definition Classes
- TraversableOnce
-
def
aggregate[B](z: ⇒ B)(seqop: (B, StructField) ⇒ B, combop: (B, B) ⇒ B): B
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
andThen[C](k: (StructField) ⇒ C): PartialFunction[Int, C]
- Definition Classes
- PartialFunction → Function1
-
def
apply(fieldIndex: Int): StructField
- Definition Classes
- StructType → SeqLike → GenSeqLike → Function1
-
def
apply(names: Set[String]): StructType
Returns a StructType containing StructFields of the given names, preserving the original order of fields.
Returns a StructType containing StructFields of the given names, preserving the original order of fields.
- Exceptions thrown
IllegalArgumentException
if at least one given field name does not exist
-
def
apply(name: String): StructField
Extracts the StructField with the given name.
Extracts the StructField with the given name.
- Exceptions thrown
IllegalArgumentException
if a field with the given name does not exist
-
def
applyOrElse[A1 <: Int, B1 >: StructField](x: A1, default: (A1) ⇒ B1): B1
- Definition Classes
- PartialFunction
-
final
def
asInstanceOf[T0]: T0
- Definition Classes
- Any
-
def
canEqual(that: Any): Boolean
- Definition Classes
- IterableLike → Equals
-
def
catalogString: String
String representation for the type saved in external catalogs.
String representation for the type saved in external catalogs.
- Definition Classes
- StructType → DataType
-
def
clone(): AnyRef
- Attributes
- protected[lang]
- Definition Classes
- AnyRef
- Annotations
- @throws( ... ) @native()
-
def
collect[B, That](pf: PartialFunction[StructField, B])(implicit bf: CanBuildFrom[Seq[StructField], B, That]): That
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
collectFirst[B](pf: PartialFunction[StructField, B]): Option[B]
- Definition Classes
- TraversableOnce
-
def
combinations(n: Int): Iterator[Seq[StructField]]
- Definition Classes
- SeqLike
-
def
companion: GenericCompanion[Seq]
- Definition Classes
- Seq → GenSeq → Iterable → GenIterable → Traversable → GenTraversable → GenericTraversableTemplate
-
def
compose[A](g: (A) ⇒ Int): (A) ⇒ StructField
- Definition Classes
- Function1
- Annotations
- @unspecialized()
-
def
contains[A1 >: StructField](elem: A1): Boolean
- Definition Classes
- SeqLike
-
def
containsSlice[B](that: GenSeq[B]): Boolean
- Definition Classes
- SeqLike
-
def
copyToArray[B >: StructField](xs: Array[B], start: Int, len: Int): Unit
- Definition Classes
- IterableLike → TraversableLike → TraversableOnce → GenTraversableOnce
-
def
copyToArray[B >: StructField](xs: Array[B]): Unit
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
copyToArray[B >: StructField](xs: Array[B], start: Int): Unit
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
copyToBuffer[B >: StructField](dest: Buffer[B]): Unit
- Definition Classes
- TraversableOnce
-
def
corresponds[B](that: GenSeq[B])(p: (StructField, B) ⇒ Boolean): Boolean
- Definition Classes
- SeqLike → GenSeqLike
-
def
count(p: (StructField) ⇒ Boolean): Int
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
defaultSize: Int
The default size of a value of the StructType is the total default sizes of all field types.
The default size of a value of the StructType is the total default sizes of all field types.
- Definition Classes
- StructType → DataType
-
def
diff[B >: StructField](that: GenSeq[B]): Seq[StructField]
- Definition Classes
- SeqLike → GenSeqLike
-
def
distinct: Seq[StructField]
- Definition Classes
- SeqLike → GenSeqLike
-
def
drop(n: Int): Seq[StructField]
- Definition Classes
- IterableLike → TraversableLike → GenTraversableLike
-
def
dropRight(n: Int): Seq[StructField]
- Definition Classes
- IterableLike
-
def
dropWhile(p: (StructField) ⇒ Boolean): Seq[StructField]
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
endsWith[B](that: GenSeq[B]): Boolean
- Definition Classes
- SeqLike → GenSeqLike
-
final
def
eq(arg0: AnyRef): Boolean
- Definition Classes
- AnyRef
-
def
equals(that: Any): Boolean
- Definition Classes
- StructType → GenSeqLike → Equals → AnyRef → Any
-
def
exists(p: (StructField) ⇒ Boolean): Boolean
- Definition Classes
- IterableLike → TraversableLike → TraversableOnce → GenTraversableOnce
-
def
fieldIndex(name: String): Int
Returns the index of a given field.
Returns the index of a given field.
- Exceptions thrown
IllegalArgumentException
if a field with the given name does not exist
-
def
fieldNames: Array[String]
Returns all field names in an array.
- val fields: Array[StructField]
-
def
filter(p: (StructField) ⇒ Boolean): Seq[StructField]
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
filterNot(p: (StructField) ⇒ Boolean): Seq[StructField]
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
finalize(): Unit
- Attributes
- protected[lang]
- Definition Classes
- AnyRef
- Annotations
- @throws( classOf[java.lang.Throwable] )
-
def
find(p: (StructField) ⇒ Boolean): Option[StructField]
- Definition Classes
- IterableLike → TraversableLike → TraversableOnce → GenTraversableOnce
-
def
flatMap[B, That](f: (StructField) ⇒ GenTraversableOnce[B])(implicit bf: CanBuildFrom[Seq[StructField], B, That]): That
- Definition Classes
- TraversableLike → GenTraversableLike → FilterMonadic
-
def
flatten[B](implicit asTraversable: (StructField) ⇒ GenTraversableOnce[B]): Seq[B]
- Definition Classes
- GenericTraversableTemplate
-
def
fold[A1 >: StructField](z: A1)(op: (A1, A1) ⇒ A1): A1
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
foldLeft[B](z: B)(op: (B, StructField) ⇒ B): B
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
foldRight[B](z: B)(op: (StructField, B) ⇒ B): B
- Definition Classes
- IterableLike → TraversableOnce → GenTraversableOnce
-
def
forall(p: (StructField) ⇒ Boolean): Boolean
- Definition Classes
- IterableLike → TraversableLike → TraversableOnce → GenTraversableOnce
-
def
foreach[U](f: (StructField) ⇒ U): Unit
- Definition Classes
- IterableLike → TraversableLike → GenTraversableLike → TraversableOnce → GenTraversableOnce → FilterMonadic
-
def
genericBuilder[B]: Builder[B, Seq[B]]
- Definition Classes
- GenericTraversableTemplate
-
final
def
getClass(): Class[_]
- Definition Classes
- AnyRef → Any
- Annotations
- @native()
-
def
groupBy[K](f: (StructField) ⇒ K): Map[K, Seq[StructField]]
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
grouped(size: Int): Iterator[Seq[StructField]]
- Definition Classes
- IterableLike
-
def
hasDefiniteSize: Boolean
- Definition Classes
- TraversableLike → TraversableOnce → GenTraversableOnce
-
def
hashCode(): Int
- Definition Classes
- StructType → GenSeqLike → AnyRef → Any
-
def
head: StructField
- Definition Classes
- IterableLike → TraversableLike → GenTraversableLike
-
def
headOption: Option[StructField]
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
indexOf[B >: StructField](elem: B, from: Int): Int
- Definition Classes
- GenSeqLike
-
def
indexOf[B >: StructField](elem: B): Int
- Definition Classes
- GenSeqLike
-
def
indexOfSlice[B >: StructField](that: GenSeq[B], from: Int): Int
- Definition Classes
- SeqLike
-
def
indexOfSlice[B >: StructField](that: GenSeq[B]): Int
- Definition Classes
- SeqLike
-
def
indexWhere(p: (StructField) ⇒ Boolean, from: Int): Int
- Definition Classes
- SeqLike → GenSeqLike
-
def
indexWhere(p: (StructField) ⇒ Boolean): Int
- Definition Classes
- GenSeqLike
-
def
indices: Range
- Definition Classes
- SeqLike
-
def
init: Seq[StructField]
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
inits: Iterator[Seq[StructField]]
- Definition Classes
- TraversableLike
-
def
intersect[B >: StructField](that: GenSeq[B]): Seq[StructField]
- Definition Classes
- SeqLike → GenSeqLike
-
def
isDefinedAt(idx: Int): Boolean
- Definition Classes
- GenSeqLike
-
def
isEmpty: Boolean
- Definition Classes
- SeqLike → IterableLike → TraversableLike → TraversableOnce → GenTraversableOnce
-
final
def
isInstanceOf[T0]: Boolean
- Definition Classes
- Any
-
final
def
isTraversableAgain: Boolean
- Definition Classes
- TraversableLike → GenTraversableLike → GenTraversableOnce
-
def
iterator: Iterator[StructField]
- Definition Classes
- StructType → IterableLike → GenIterableLike
-
def
json: String
The compact JSON representation of this data type.
The compact JSON representation of this data type.
- Definition Classes
- DataType
-
def
last: StructField
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
lastIndexOf[B >: StructField](elem: B, end: Int): Int
- Definition Classes
- GenSeqLike
-
def
lastIndexOf[B >: StructField](elem: B): Int
- Definition Classes
- GenSeqLike
-
def
lastIndexOfSlice[B >: StructField](that: GenSeq[B], end: Int): Int
- Definition Classes
- SeqLike
-
def
lastIndexOfSlice[B >: StructField](that: GenSeq[B]): Int
- Definition Classes
- SeqLike
-
def
lastIndexWhere(p: (StructField) ⇒ Boolean, end: Int): Int
- Definition Classes
- SeqLike → GenSeqLike
-
def
lastIndexWhere(p: (StructField) ⇒ Boolean): Int
- Definition Classes
- GenSeqLike
-
def
lastOption: Option[StructField]
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
length: Int
- Definition Classes
- StructType → SeqLike → GenSeqLike
-
def
lengthCompare(len: Int): Int
- Definition Classes
- SeqLike
-
def
lift: (Int) ⇒ Option[StructField]
- Definition Classes
- PartialFunction
-
def
map[B, That](f: (StructField) ⇒ B)(implicit bf: CanBuildFrom[Seq[StructField], B, That]): That
- Definition Classes
- TraversableLike → GenTraversableLike → FilterMonadic
-
def
max[B >: StructField](implicit cmp: Ordering[B]): StructField
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
maxBy[B](f: (StructField) ⇒ B)(implicit cmp: Ordering[B]): StructField
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
min[B >: StructField](implicit cmp: Ordering[B]): StructField
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
minBy[B](f: (StructField) ⇒ B)(implicit cmp: Ordering[B]): StructField
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
mkString: String
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
mkString(sep: String): String
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
mkString(start: String, sep: String, end: String): String
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
names: Array[String]
Returns all field names in an array.
Returns all field names in an array. This is an alias of
fieldNames
.- Since
2.4.0
-
final
def
ne(arg0: AnyRef): Boolean
- Definition Classes
- AnyRef
-
def
newBuilder: Builder[StructField, Seq[StructField]]
- Attributes
- protected[this]
- Definition Classes
- GenericTraversableTemplate → HasNewBuilder
-
def
nonEmpty: Boolean
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
final
def
notify(): Unit
- Definition Classes
- AnyRef
- Annotations
- @native()
-
final
def
notifyAll(): Unit
- Definition Classes
- AnyRef
- Annotations
- @native()
-
def
orElse[A1 <: Int, B1 >: StructField](that: PartialFunction[A1, B1]): PartialFunction[A1, B1]
- Definition Classes
- PartialFunction
-
def
padTo[B >: StructField, That](len: Int, elem: B)(implicit bf: CanBuildFrom[Seq[StructField], B, That]): That
- Definition Classes
- SeqLike → GenSeqLike
-
def
par: ParSeq[StructField]
- Definition Classes
- Parallelizable
-
def
parCombiner: Combiner[StructField, ParSeq[StructField]]
- Attributes
- protected[this]
- Definition Classes
- SeqLike → TraversableLike → Parallelizable
-
def
partition(p: (StructField) ⇒ Boolean): (Seq[StructField], Seq[StructField])
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
patch[B >: StructField, That](from: Int, patch: GenSeq[B], replaced: Int)(implicit bf: CanBuildFrom[Seq[StructField], B, That]): That
- Definition Classes
- SeqLike → GenSeqLike
-
def
permutations: Iterator[Seq[StructField]]
- Definition Classes
- SeqLike
-
def
prefixLength(p: (StructField) ⇒ Boolean): Int
- Definition Classes
- GenSeqLike
-
def
prettyJson: String
The pretty (i.e.
The pretty (i.e. indented) JSON representation of this data type.
- Definition Classes
- DataType
- def printTreeString(): Unit
-
def
product[B >: StructField](implicit num: Numeric[B]): B
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
reduce[A1 >: StructField](op: (A1, A1) ⇒ A1): A1
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
reduceLeft[B >: StructField](op: (B, StructField) ⇒ B): B
- Definition Classes
- TraversableOnce
-
def
reduceLeftOption[B >: StructField](op: (B, StructField) ⇒ B): Option[B]
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
reduceOption[A1 >: StructField](op: (A1, A1) ⇒ A1): Option[A1]
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
reduceRight[B >: StructField](op: (StructField, B) ⇒ B): B
- Definition Classes
- IterableLike → TraversableOnce → GenTraversableOnce
-
def
reduceRightOption[B >: StructField](op: (StructField, B) ⇒ B): Option[B]
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
repr: Seq[StructField]
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
reverse: Seq[StructField]
- Definition Classes
- SeqLike → GenSeqLike
-
def
reverseIterator: Iterator[StructField]
- Definition Classes
- SeqLike
-
def
reverseMap[B, That](f: (StructField) ⇒ B)(implicit bf: CanBuildFrom[Seq[StructField], B, That]): That
- Definition Classes
- SeqLike → GenSeqLike
-
def
reversed: List[StructField]
- Attributes
- protected[this]
- Definition Classes
- TraversableOnce
-
def
runWith[U](action: (StructField) ⇒ U): (Int) ⇒ Boolean
- Definition Classes
- PartialFunction
-
def
sameElements[B >: StructField](that: GenIterable[B]): Boolean
- Definition Classes
- IterableLike → GenIterableLike
-
def
scan[B >: StructField, That](z: B)(op: (B, B) ⇒ B)(implicit cbf: CanBuildFrom[Seq[StructField], B, That]): That
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
scanLeft[B, That](z: B)(op: (B, StructField) ⇒ B)(implicit bf: CanBuildFrom[Seq[StructField], B, That]): That
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
scanRight[B, That](z: B)(op: (StructField, B) ⇒ B)(implicit bf: CanBuildFrom[Seq[StructField], B, That]): That
- Definition Classes
- TraversableLike → GenTraversableLike
- Annotations
- @migration
- Migration
(Changed in version 2.9.0) The behavior of
scanRight
has changed. The previous behavior can be reproduced with scanRight.reverse.
-
def
segmentLength(p: (StructField) ⇒ Boolean, from: Int): Int
- Definition Classes
- SeqLike → GenSeqLike
-
def
seq: Seq[StructField]
- Definition Classes
- Seq → GenSeq → GenSeqLike → Iterable → GenIterable → Traversable → GenTraversable → Parallelizable → TraversableOnce → GenTraversableOnce
-
def
simpleString: String
Readable string representation for the type.
Readable string representation for the type.
- Definition Classes
- StructType → DataType → AbstractDataType
-
def
size: Int
- Definition Classes
- SeqLike → GenTraversableLike → TraversableOnce → GenTraversableOnce
-
def
sizeHintIfCheap: Int
- Attributes
- protected[collection]
- Definition Classes
- GenTraversableOnce
-
def
slice(from: Int, until: Int): Seq[StructField]
- Definition Classes
- IterableLike → TraversableLike → GenTraversableLike
-
def
sliding(size: Int, step: Int): Iterator[Seq[StructField]]
- Definition Classes
- IterableLike
-
def
sliding(size: Int): Iterator[Seq[StructField]]
- Definition Classes
- IterableLike
-
def
sortBy[B](f: (StructField) ⇒ B)(implicit ord: Ordering[B]): Seq[StructField]
- Definition Classes
- SeqLike
-
def
sortWith(lt: (StructField, StructField) ⇒ Boolean): Seq[StructField]
- Definition Classes
- SeqLike
-
def
sorted[B >: StructField](implicit ord: Ordering[B]): Seq[StructField]
- Definition Classes
- SeqLike
-
def
span(p: (StructField) ⇒ Boolean): (Seq[StructField], Seq[StructField])
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
splitAt(n: Int): (Seq[StructField], Seq[StructField])
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
sql: String
- Definition Classes
- StructType → DataType
-
def
startsWith[B](that: GenSeq[B], offset: Int): Boolean
- Definition Classes
- SeqLike → GenSeqLike
-
def
startsWith[B](that: GenSeq[B]): Boolean
- Definition Classes
- GenSeqLike
-
def
stringPrefix: String
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
sum[B >: StructField](implicit num: Numeric[B]): B
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
final
def
synchronized[T0](arg0: ⇒ T0): T0
- Definition Classes
- AnyRef
-
def
tail: Seq[StructField]
- Definition Classes
- TraversableLike → GenTraversableLike
-
def
tails: Iterator[Seq[StructField]]
- Definition Classes
- TraversableLike
-
def
take(n: Int): Seq[StructField]
- Definition Classes
- IterableLike → TraversableLike → GenTraversableLike
-
def
takeRight(n: Int): Seq[StructField]
- Definition Classes
- IterableLike
-
def
takeWhile(p: (StructField) ⇒ Boolean): Seq[StructField]
- Definition Classes
- IterableLike → TraversableLike → GenTraversableLike
-
def
thisCollection: Seq[StructField]
- Attributes
- protected[this]
- Definition Classes
- SeqLike → IterableLike → TraversableLike
-
def
to[Col[_]](implicit cbf: CanBuildFrom[Nothing, StructField, Col[StructField]]): Col[StructField]
- Definition Classes
- TraversableLike → TraversableOnce → GenTraversableOnce
-
def
toArray[B >: StructField](implicit arg0: ClassTag[B]): Array[B]
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
toBuffer[B >: StructField]: Buffer[B]
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
toCollection(repr: Seq[StructField]): Seq[StructField]
- Attributes
- protected[this]
- Definition Classes
- SeqLike → IterableLike → TraversableLike
-
def
toDDL: String
Returns a string containing a schema in DDL format.
Returns a string containing a schema in DDL format. For example, the following value:
StructType(Seq(StructField("eventId", IntegerType), StructField("s", StringType)))
will be converted toeventId
INT,s
STRING. The returned DDL schema can be used in a table creation.- Since
2.4.0
-
def
toIndexedSeq: IndexedSeq[StructField]
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
toIterable: Iterable[StructField]
- Definition Classes
- IterableLike → TraversableOnce → GenTraversableOnce
-
def
toIterator: Iterator[StructField]
- Definition Classes
- IterableLike → TraversableLike → GenTraversableOnce
- Annotations
- @deprecatedOverriding( ... , "2.11.0" )
-
def
toList: List[StructField]
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
toMap[T, U](implicit ev: <:<[StructField, (T, U)]): Map[T, U]
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
toSeq: Seq[StructField]
- Definition Classes
- SeqLike → GenSeqLike → TraversableOnce → GenTraversableOnce
-
def
toSet[B >: StructField]: Set[B]
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
toStream: Stream[StructField]
- Definition Classes
- IterableLike → TraversableLike → GenTraversableOnce
-
def
toString(): String
- Definition Classes
- StructType → SeqLike → TraversableLike → Function1 → AnyRef → Any
-
def
toTraversable: Traversable[StructField]
- Definition Classes
- TraversableLike → TraversableOnce → GenTraversableOnce
- Annotations
- @deprecatedOverriding( ... , "2.11.0" )
-
def
toVector: Vector[StructField]
- Definition Classes
- TraversableOnce → GenTraversableOnce
-
def
transpose[B](implicit asTraversable: (StructField) ⇒ GenTraversableOnce[B]): Seq[Seq[B]]
- Definition Classes
- GenericTraversableTemplate
- Annotations
- @migration
- Migration
(Changed in version 2.9.0)
transpose
throws anIllegalArgumentException
if collections are not uniformly sized.
- def treeString(maxDepth: Int): String
- def treeString: String
-
def
typeName: String
Name of the type used in JSON serialization.
Name of the type used in JSON serialization.
- Definition Classes
- DataType
-
def
union[B >: StructField, That](that: GenSeq[B])(implicit bf: CanBuildFrom[Seq[StructField], B, That]): That
- Definition Classes
- SeqLike → GenSeqLike
-
def
unzip[A1, A2](implicit asPair: (StructField) ⇒ (A1, A2)): (Seq[A1], Seq[A2])
- Definition Classes
- GenericTraversableTemplate
-
def
unzip3[A1, A2, A3](implicit asTriple: (StructField) ⇒ (A1, A2, A3)): (Seq[A1], Seq[A2], Seq[A3])
- Definition Classes
- GenericTraversableTemplate
-
def
updated[B >: StructField, That](index: Int, elem: B)(implicit bf: CanBuildFrom[Seq[StructField], B, That]): That
- Definition Classes
- SeqLike → GenSeqLike
-
def
view(from: Int, until: Int): SeqView[StructField, Seq[StructField]]
- Definition Classes
- SeqLike → IterableLike → TraversableLike
-
def
view: SeqView[StructField, Seq[StructField]]
- Definition Classes
- SeqLike → IterableLike → TraversableLike
-
final
def
wait(): Unit
- Definition Classes
- AnyRef
- Annotations
- @throws( ... )
-
final
def
wait(arg0: Long, arg1: Int): Unit
- Definition Classes
- AnyRef
- Annotations
- @throws( ... )
-
final
def
wait(arg0: Long): Unit
- Definition Classes
- AnyRef
- Annotations
- @throws( ... ) @native()
-
def
withFilter(p: (StructField) ⇒ Boolean): FilterMonadic[StructField, Seq[StructField]]
- Definition Classes
- TraversableLike → FilterMonadic
-
def
zip[A1 >: StructField, B, That](that: GenIterable[B])(implicit bf: CanBuildFrom[Seq[StructField], (A1, B), That]): That
- Definition Classes
- IterableLike → GenIterableLike
-
def
zipAll[B, A1 >: StructField, That](that: GenIterable[B], thisElem: A1, thatElem: B)(implicit bf: CanBuildFrom[Seq[StructField], (A1, B), That]): That
- Definition Classes
- IterableLike → GenIterableLike
-
def
zipWithIndex[A1 >: StructField, That](implicit bf: CanBuildFrom[Seq[StructField], (A1, Int), That]): That
- Definition Classes
- IterableLike → GenIterableLike
Deprecated Value Members
-
def
/:[B](z: B)(op: (B, StructField) ⇒ B): B
- Definition Classes
- TraversableOnce → GenTraversableOnce
- Annotations
- @deprecated
- Deprecated
(Since version 2.12.10) Use foldLeft instead of /:
-
def
:\[B](z: B)(op: (StructField, B) ⇒ B): B
- Definition Classes
- TraversableOnce → GenTraversableOnce
- Annotations
- @deprecated
- Deprecated
(Since version 2.12.10) Use foldRight instead of :\