pyspark.sql.types.
StructType
Struct type, consisting of a list of StructField.
StructField
This is the data type representing a Row.
Row
Iterating a StructType will iterate over its StructFields. A contained StructField can be accessed by its name or position.
Examples
>>> from pyspark.sql.types import * >>> struct1 = StructType([StructField("f1", StringType(), True)]) >>> struct1["f1"] StructField('f1', StringType(), True) >>> struct1[0] StructField('f1', StringType(), True)
>>> struct1 = StructType([StructField("f1", StringType(), True)]) >>> struct2 = StructType([StructField("f1", StringType(), True)]) >>> struct1 == struct2 True >>> struct1 = StructType([StructField("f1", CharType(10), True)]) >>> struct2 = StructType([StructField("f1", CharType(10), True)]) >>> struct1 == struct2 True >>> struct1 = StructType([StructField("f1", VarcharType(10), True)]) >>> struct2 = StructType([StructField("f1", VarcharType(10), True)]) >>> struct1 == struct2 True >>> struct1 = StructType([StructField("f1", StringType(), True)]) >>> struct2 = StructType([StructField("f1", StringType(), True), ... StructField("f2", IntegerType(), False)]) >>> struct1 == struct2 False
The below example demonstrates how to create a DataFrame based on a struct created using class:StructType and class:StructField:
>>> data = [("Alice", ["Java", "Scala"]), ("Bob", ["Python", "Scala"])] >>> schema = StructType([ ... StructField("name", StringType()), ... StructField("languagesSkills", ArrayType(StringType())), ... ]) >>> df = spark.createDataFrame(data=data, schema=schema) >>> df.printSchema() root |-- name: string (nullable = true) |-- languagesSkills: array (nullable = true) | |-- element: string (containsNull = true) >>> df.show() +-----+---------------+ | name|languagesSkills| +-----+---------------+ |Alice| [Java, Scala]| | Bob|[Python, Scala]| +-----+---------------+
Methods
add(field[, data_type, nullable, metadata])
add
Construct a StructType by adding new elements to it, to define the schema.
fieldNames()
fieldNames
Returns all field names in a list.
fromInternal(obj)
fromInternal
Converts an internal SQL object into a native Python object.
fromJson(json)
fromJson
Constructs StructType from a schema defined in JSON format.
json()
json
jsonValue()
jsonValue
needConversion()
needConversion
Does this type needs conversion between Python object and internal SQL object.
simpleString()
simpleString
toInternal(obj)
toInternal
Converts a Python object into an internal SQL object.
typeName()
typeName
Methods Documentation
Construct a StructType by adding new elements to it, to define the schema. The method accepts either:
A single parameter which is a StructField object. Between 2 and 4 parameters as (name, data_type, nullable (optional), metadata(optional). The data_type parameter may be either a String or a DataType object.
A single parameter which is a StructField object.
Between 2 and 4 parameters as (name, data_type, nullable (optional), metadata(optional). The data_type parameter may be either a String or a DataType object.
DataType
Either the name of the field or a StructField object
If present, the DataType of the StructField to create
Whether the field to add should be nullable (default True)
Any additional metadata (default None)
>>> from pyspark.sql.types import IntegerType, StringType, StructField, StructType >>> struct1 = StructType().add("f1", StringType(), True).add("f2", StringType(), True, None) >>> struct2 = StructType([StructField("f1", StringType(), True), ... StructField("f2", StringType(), True, None)]) >>> struct1 == struct2 True >>> struct1 = StructType().add(StructField("f1", StringType(), True)) >>> struct2 = StructType([StructField("f1", StringType(), True)]) >>> struct1 == struct2 True >>> struct1 = StructType().add("f1", "string", True) >>> struct2 = StructType([StructField("f1", StringType(), True)]) >>> struct1 == struct2 True
>>> from pyspark.sql.types import StringType, StructField, StructType >>> struct = StructType([StructField("f1", StringType(), True)]) >>> struct.fieldNames() ['f1']
Below is a JSON schema it must adhere to:
{ "title":"StructType", "description":"Schema of StructType in json format", "type":"object", "properties":{ "fields":{ "description":"Array of struct fields", "type":"array", "items":{ "type":"object", "properties":{ "name":{ "description":"Name of the field", "type":"string" }, "type":{ "description": "Type of the field. Can either be another nested StructType or primitive type", "type":"object/string" }, "nullable":{ "description":"If nulls are allowed", "type":"boolean" }, "metadata":{ "description":"Additional metadata to supply", "type":"object" }, "required":[ "name", "type", "nullable", "metadata" ] } } } } }
This “dict” must have “fields” key that returns an array of fields each of which must have specific keys (name, type, nullable, metadata).
>>> json_str = ''' ... { ... "fields": [ ... { ... "metadata": {}, ... "name": "Person", ... "nullable": true, ... "type": { ... "fields": [ ... { ... "metadata": {}, ... "name": "name", ... "nullable": false, ... "type": "string" ... }, ... { ... "metadata": {}, ... "name": "surname", ... "nullable": false, ... "type": "string" ... } ... ], ... "type": "struct" ... } ... } ... ], ... "type": "struct" ... } ... ''' >>> import json >>> scheme = StructType.fromJson(json.loads(json_str)) >>> scheme.simpleString() 'struct<Person:struct<name:string,surname:string>>'
This is used to avoid the unnecessary conversion for ArrayType/MapType/StructType.