| |
| <!DOCTYPE html> |
| <!--[if lt IE 7]> <html class="no-js lt-ie9 lt-ie8 lt-ie7"> <![endif]--> |
| <!--[if IE 7]> <html class="no-js lt-ie9 lt-ie8"> <![endif]--> |
| <!--[if IE 8]> <html class="no-js lt-ie9"> <![endif]--> |
| <!--[if gt IE 8]><!--> <html class="no-js"> <!--<![endif]--> |
| <head> |
| <meta charset="utf-8"> |
| <meta http-equiv="X-UA-Compatible" content="IE=edge,chrome=1"> |
| <title>JSON Files - Spark 3.3.4 Documentation</title> |
| |
| |
| |
| |
| <link rel="stylesheet" href="css/bootstrap.min.css"> |
| <style> |
| body { |
| padding-top: 60px; |
| padding-bottom: 40px; |
| } |
| </style> |
| <meta name="viewport" content="width=device-width"> |
| <link rel="stylesheet" href="css/main.css"> |
| |
| <script src="js/vendor/modernizr-2.6.1-respond-1.1.0.min.js"></script> |
| |
| <link rel="stylesheet" href="css/pygments-default.css"> |
| <link rel="stylesheet" href="https://cdn.jsdelivr.net/npm/docsearch.js@2/dist/cdn/docsearch.min.css" /> |
| <link rel="stylesheet" href="css/docsearch.css"> |
| |
| |
| <!-- Matomo --> |
| <script> |
| var _paq = window._paq = window._paq || []; |
| /* tracker methods like "setCustomDimension" should be called before "trackPageView" */ |
| _paq.push(["disableCookies"]); |
| _paq.push(['trackPageView']); |
| _paq.push(['enableLinkTracking']); |
| (function() { |
| var u="https://analytics.apache.org/"; |
| _paq.push(['setTrackerUrl', u+'matomo.php']); |
| _paq.push(['setSiteId', '40']); |
| var d=document, g=d.createElement('script'), s=d.getElementsByTagName('script')[0]; |
| g.async=true; g.src=u+'matomo.js'; s.parentNode.insertBefore(g,s); |
| })(); |
| </script> |
| <!-- End Matomo Code --> |
| |
| |
| </head> |
| <body> |
| <!--[if lt IE 7]> |
| <p class="chromeframe">You are using an outdated browser. <a href="https://browsehappy.com/">Upgrade your browser today</a> or <a href="http://www.google.com/chromeframe/?redirect=true">install Google Chrome Frame</a> to better experience this site.</p> |
| <![endif]--> |
| |
| <!-- This code is taken from http://twitter.github.com/bootstrap/examples/hero.html --> |
| |
| <nav class="navbar fixed-top navbar-expand-md navbar-light bg-light" id="topbar"> |
| <div class="container"> |
| <div class="navbar-header"> |
| <div class="navbar-brand"><a href="index.html"> |
| <img src="img/spark-logo-hd.png" style="height:50px;"/></a><span class="version">3.3.4</span> |
| </div> |
| </div> |
| <button class="navbar-toggler" type="button" data-toggle="collapse" |
| data-target="#navbarCollapse" aria-controls="navbarCollapse" |
| aria-expanded="false" aria-label="Toggle navigation"> |
| <span class="navbar-toggler-icon"></span> |
| </button> |
| <div class="collapse navbar-collapse" id="navbarCollapse"> |
| <ul class="navbar-nav"> |
| <!--TODO(andyk): Add class="active" attribute to li some how.--> |
| <li class="nav-item"><a href="index.html" class="nav-link">Overview</a></li> |
| |
| <li class="nav-item dropdown"> |
| <a href="#" class="nav-link dropdown-toggle" id="navbarQuickStart" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">Programming Guides</a> |
| <div class="dropdown-menu" aria-labelledby="navbarQuickStart"> |
| <a class="dropdown-item" href="quick-start.html">Quick Start</a> |
| <a class="dropdown-item" href="rdd-programming-guide.html">RDDs, Accumulators, Broadcasts Vars</a> |
| <a class="dropdown-item" href="sql-programming-guide.html">SQL, DataFrames, and Datasets</a> |
| <a class="dropdown-item" href="structured-streaming-programming-guide.html">Structured Streaming</a> |
| <a class="dropdown-item" href="streaming-programming-guide.html">Spark Streaming (DStreams)</a> |
| <a class="dropdown-item" href="ml-guide.html">MLlib (Machine Learning)</a> |
| <a class="dropdown-item" href="graphx-programming-guide.html">GraphX (Graph Processing)</a> |
| <a class="dropdown-item" href="sparkr.html">SparkR (R on Spark)</a> |
| <a class="dropdown-item" href="api/python/getting_started/index.html">PySpark (Python on Spark)</a> |
| </div> |
| </li> |
| |
| <li class="nav-item dropdown"> |
| <a href="#" class="nav-link dropdown-toggle" id="navbarAPIDocs" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">API Docs</a> |
| <div class="dropdown-menu" aria-labelledby="navbarAPIDocs"> |
| <a class="dropdown-item" href="api/scala/org/apache/spark/index.html">Scala</a> |
| <a class="dropdown-item" href="api/java/index.html">Java</a> |
| <a class="dropdown-item" href="api/python/index.html">Python</a> |
| <a class="dropdown-item" href="api/R/index.html">R</a> |
| <a class="dropdown-item" href="api/sql/index.html">SQL, Built-in Functions</a> |
| </div> |
| </li> |
| |
| <li class="nav-item dropdown"> |
| <a href="#" class="nav-link dropdown-toggle" id="navbarDeploying" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">Deploying</a> |
| <div class="dropdown-menu" aria-labelledby="navbarDeploying"> |
| <a class="dropdown-item" href="cluster-overview.html">Overview</a> |
| <a class="dropdown-item" href="submitting-applications.html">Submitting Applications</a> |
| <div class="dropdown-divider"></div> |
| <a class="dropdown-item" href="spark-standalone.html">Spark Standalone</a> |
| <a class="dropdown-item" href="running-on-mesos.html">Mesos</a> |
| <a class="dropdown-item" href="running-on-yarn.html">YARN</a> |
| <a class="dropdown-item" href="running-on-kubernetes.html">Kubernetes</a> |
| </div> |
| </li> |
| |
| <li class="nav-item dropdown"> |
| <a href="#" class="nav-link dropdown-toggle" id="navbarMore" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">More</a> |
| <div class="dropdown-menu" aria-labelledby="navbarMore"> |
| <a class="dropdown-item" href="configuration.html">Configuration</a> |
| <a class="dropdown-item" href="monitoring.html">Monitoring</a> |
| <a class="dropdown-item" href="tuning.html">Tuning Guide</a> |
| <a class="dropdown-item" href="job-scheduling.html">Job Scheduling</a> |
| <a class="dropdown-item" href="security.html">Security</a> |
| <a class="dropdown-item" href="hardware-provisioning.html">Hardware Provisioning</a> |
| <a class="dropdown-item" href="migration-guide.html">Migration Guide</a> |
| <div class="dropdown-divider"></div> |
| <a class="dropdown-item" href="building-spark.html">Building Spark</a> |
| <a class="dropdown-item" href="https://spark.apache.org/contributing.html">Contributing to Spark</a> |
| <a class="dropdown-item" href="https://spark.apache.org/third-party-projects.html">Third Party Projects</a> |
| </div> |
| </li> |
| |
| <li class="nav-item"> |
| <input type="text" id="docsearch-input" placeholder="Search the docs…"> |
| </li> |
| </ul> |
| <!--<span class="navbar-text navbar-right"><span class="version-text">v3.3.4</span></span>--> |
| </div> |
| </div> |
| </nav> |
| |
| <div class="container-wrapper"> |
| |
| |
| |
| <div class="left-menu-wrapper"> |
| <div class="left-menu"> |
| <h3><a href="sql-programming-guide.html">Spark SQL Guide</a></h3> |
| |
| <ul> |
| |
| <li> |
| <a href="sql-getting-started.html"> |
| |
| Getting Started |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-data-sources.html"> |
| |
| Data Sources |
| |
| </a> |
| </li> |
| |
| |
| |
| <ul> |
| |
| <li> |
| <a href="sql-data-sources-load-save-functions.html"> |
| |
| Generic Load/Save Functions |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-data-sources-generic-options.html"> |
| |
| Generic File Source Options |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-data-sources-parquet.html"> |
| |
| Parquet Files |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-data-sources-orc.html"> |
| |
| ORC Files |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-data-sources-json.html"> |
| |
| JSON Files |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-data-sources-csv.html"> |
| |
| CSV Files |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-data-sources-text.html"> |
| |
| Text Files |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-data-sources-hive-tables.html"> |
| |
| Hive Tables |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-data-sources-jdbc.html"> |
| |
| JDBC To Other Databases |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-data-sources-avro.html"> |
| |
| Avro Files |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-data-sources-binaryFile.html"> |
| |
| Whole Binary Files |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-data-sources-troubleshooting.html"> |
| |
| Troubleshooting |
| |
| </a> |
| </li> |
| |
| |
| |
| </ul> |
| |
| |
| |
| <li> |
| <a href="sql-performance-tuning.html"> |
| |
| Performance Tuning |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-distributed-sql-engine.html"> |
| |
| Distributed SQL Engine |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-pyspark-pandas-with-arrow.html"> |
| |
| PySpark Usage Guide for Pandas with Apache Arrow |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-migration-old.html"> |
| |
| Migration Guide |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-ref.html"> |
| |
| SQL Reference |
| |
| </a> |
| </li> |
| |
| |
| |
| </ul> |
| |
| </div> |
| </div> |
| |
| <input id="nav-trigger" class="nav-trigger" checked type="checkbox"> |
| <label for="nav-trigger"></label> |
| <div class="content-with-sidebar mr-3" id="content"> |
| |
| <h1 class="title">JSON Files</h1> |
| |
| |
| <div class="codetabs"> |
| |
| <div data-lang="scala"> |
| <p>Spark SQL can automatically infer the schema of a JSON dataset and load it as a <code class="language-plaintext highlighter-rouge">Dataset[Row]</code>. |
| This conversion can be done using <code class="language-plaintext highlighter-rouge">SparkSession.read.json()</code> on either a <code class="language-plaintext highlighter-rouge">Dataset[String]</code>, |
| or a JSON file.</p> |
| |
| <p>Note that the file that is offered as <em>a json file</em> is not a typical JSON file. Each |
| line must contain a separate, self-contained valid JSON object. For more information, please see |
| <a href="http://jsonlines.org/">JSON Lines text format, also called newline-delimited JSON</a>.</p> |
| |
| <p>For a regular multi-line JSON file, set the <code class="language-plaintext highlighter-rouge">multiLine</code> option to <code class="language-plaintext highlighter-rouge">true</code>.</p> |
| |
| <div class="highlight"><pre class="codehilite"><code><span class="c1">// Primitive types (Int, String, etc) and Product types (case classes) encoders are</span> |
| <span class="c1">// supported by importing this when creating a Dataset.</span> |
| <span class="k">import</span> <span class="nn">spark.implicits._</span> |
| |
| <span class="c1">// A JSON dataset is pointed to by path.</span> |
| <span class="c1">// The path can be either a single text file or a directory storing text files</span> |
| <span class="k">val</span> <span class="nv">path</span> <span class="k">=</span> <span class="s">"examples/src/main/resources/people.json"</span> |
| <span class="k">val</span> <span class="nv">peopleDF</span> <span class="k">=</span> <span class="nv">spark</span><span class="o">.</span><span class="py">read</span><span class="o">.</span><span class="py">json</span><span class="o">(</span><span class="n">path</span><span class="o">)</span> |
| |
| <span class="c1">// The inferred schema can be visualized using the printSchema() method</span> |
| <span class="nv">peopleDF</span><span class="o">.</span><span class="py">printSchema</span><span class="o">()</span> |
| <span class="c1">// root</span> |
| <span class="c1">// |-- age: long (nullable = true)</span> |
| <span class="c1">// |-- name: string (nullable = true)</span> |
| |
| <span class="c1">// Creates a temporary view using the DataFrame</span> |
| <span class="nv">peopleDF</span><span class="o">.</span><span class="py">createOrReplaceTempView</span><span class="o">(</span><span class="s">"people"</span><span class="o">)</span> |
| |
| <span class="c1">// SQL statements can be run by using the sql methods provided by spark</span> |
| <span class="k">val</span> <span class="nv">teenagerNamesDF</span> <span class="k">=</span> <span class="nv">spark</span><span class="o">.</span><span class="py">sql</span><span class="o">(</span><span class="s">"SELECT name FROM people WHERE age BETWEEN 13 AND 19"</span><span class="o">)</span> |
| <span class="nv">teenagerNamesDF</span><span class="o">.</span><span class="py">show</span><span class="o">()</span> |
| <span class="c1">// +------+</span> |
| <span class="c1">// | name|</span> |
| <span class="c1">// +------+</span> |
| <span class="c1">// |Justin|</span> |
| <span class="c1">// +------+</span> |
| |
| <span class="c1">// Alternatively, a DataFrame can be created for a JSON dataset represented by</span> |
| <span class="c1">// a Dataset[String] storing one JSON object per string</span> |
| <span class="k">val</span> <span class="nv">otherPeopleDataset</span> <span class="k">=</span> <span class="nv">spark</span><span class="o">.</span><span class="py">createDataset</span><span class="o">(</span> |
| <span class="s">"""{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}"""</span> <span class="o">::</span> <span class="nc">Nil</span><span class="o">)</span> |
| <span class="k">val</span> <span class="nv">otherPeople</span> <span class="k">=</span> <span class="nv">spark</span><span class="o">.</span><span class="py">read</span><span class="o">.</span><span class="py">json</span><span class="o">(</span><span class="n">otherPeopleDataset</span><span class="o">)</span> |
| <span class="nv">otherPeople</span><span class="o">.</span><span class="py">show</span><span class="o">()</span> |
| <span class="c1">// +---------------+----+</span> |
| <span class="c1">// | address|name|</span> |
| <span class="c1">// +---------------+----+</span> |
| <span class="c1">// |[Columbus,Ohio]| Yin|</span> |
| <span class="c1">// +---------------+----+</span></code></pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <p>Spark SQL can automatically infer the schema of a JSON dataset and load it as a <code class="language-plaintext highlighter-rouge">Dataset<Row></code>. |
| This conversion can be done using <code class="language-plaintext highlighter-rouge">SparkSession.read().json()</code> on either a <code class="language-plaintext highlighter-rouge">Dataset<String></code>, |
| or a JSON file.</p> |
| |
| <p>Note that the file that is offered as <em>a json file</em> is not a typical JSON file. Each |
| line must contain a separate, self-contained valid JSON object. For more information, please see |
| <a href="http://jsonlines.org/">JSON Lines text format, also called newline-delimited JSON</a>.</p> |
| |
| <p>For a regular multi-line JSON file, set the <code class="language-plaintext highlighter-rouge">multiLine</code> option to <code class="language-plaintext highlighter-rouge">true</code>.</p> |
| |
| <div class="highlight"><pre class="codehilite"><code><span class="kn">import</span> <span class="nn">org.apache.spark.sql.Dataset</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Row</span><span class="o">;</span> |
| |
| <span class="c1">// A JSON dataset is pointed to by path.</span> |
| <span class="c1">// The path can be either a single text file or a directory storing text files</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">people</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span><span class="na">json</span><span class="o">(</span><span class="s">"examples/src/main/resources/people.json"</span><span class="o">);</span> |
| |
| <span class="c1">// The inferred schema can be visualized using the printSchema() method</span> |
| <span class="n">people</span><span class="o">.</span><span class="na">printSchema</span><span class="o">();</span> |
| <span class="c1">// root</span> |
| <span class="c1">// |-- age: long (nullable = true)</span> |
| <span class="c1">// |-- name: string (nullable = true)</span> |
| |
| <span class="c1">// Creates a temporary view using the DataFrame</span> |
| <span class="n">people</span><span class="o">.</span><span class="na">createOrReplaceTempView</span><span class="o">(</span><span class="s">"people"</span><span class="o">);</span> |
| |
| <span class="c1">// SQL statements can be run by using the sql methods provided by spark</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">namesDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"SELECT name FROM people WHERE age BETWEEN 13 AND 19"</span><span class="o">);</span> |
| <span class="n">namesDF</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +------+</span> |
| <span class="c1">// | name|</span> |
| <span class="c1">// +------+</span> |
| <span class="c1">// |Justin|</span> |
| <span class="c1">// +------+</span> |
| |
| <span class="c1">// Alternatively, a DataFrame can be created for a JSON dataset represented by</span> |
| <span class="c1">// a Dataset<String> storing one JSON object per string.</span> |
| <span class="nc">List</span><span class="o"><</span><span class="nc">String</span><span class="o">></span> <span class="n">jsonData</span> <span class="o">=</span> <span class="nc">Arrays</span><span class="o">.</span><span class="na">asList</span><span class="o">(</span> |
| <span class="s">"{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"</span><span class="o">);</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">String</span><span class="o">></span> <span class="n">anotherPeopleDataset</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">createDataset</span><span class="o">(</span><span class="n">jsonData</span><span class="o">,</span> <span class="nc">Encoders</span><span class="o">.</span><span class="na">STRING</span><span class="o">());</span> |
| <span class="nc">Dataset</span><span class="o"><</span><span class="nc">Row</span><span class="o">></span> <span class="n">anotherPeople</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span><span class="na">json</span><span class="o">(</span><span class="n">anotherPeopleDataset</span><span class="o">);</span> |
| <span class="n">anotherPeople</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +---------------+----+</span> |
| <span class="c1">// | address|name|</span> |
| <span class="c1">// +---------------+----+</span> |
| <span class="c1">// |[Columbus,Ohio]| Yin|</span> |
| <span class="c1">// +---------------+----+</span></code></pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| <p>Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. |
| This conversion can be done using <code class="language-plaintext highlighter-rouge">SparkSession.read.json</code> on a JSON file.</p> |
| |
| <p>Note that the file that is offered as <em>a json file</em> is not a typical JSON file. Each |
| line must contain a separate, self-contained valid JSON object. For more information, please see |
| <a href="http://jsonlines.org/">JSON Lines text format, also called newline-delimited JSON</a>.</p> |
| |
| <p>For a regular multi-line JSON file, set the <code class="language-plaintext highlighter-rouge">multiLine</code> parameter to <code class="language-plaintext highlighter-rouge">True</code>.</p> |
| |
| <div class="highlight"><pre class="codehilite"><code><span class="c1"># spark is from the previous example. |
| </span><span class="n">sc</span> <span class="o">=</span> <span class="n">spark</span><span class="p">.</span><span class="n">sparkContext</span> |
| |
| <span class="c1"># A JSON dataset is pointed to by path. |
| # The path can be either a single text file or a directory storing text files |
| </span><span class="n">path</span> <span class="o">=</span> <span class="s">"examples/src/main/resources/people.json"</span> |
| <span class="n">peopleDF</span> <span class="o">=</span> <span class="n">spark</span><span class="p">.</span><span class="n">read</span><span class="p">.</span><span class="n">json</span><span class="p">(</span><span class="n">path</span><span class="p">)</span> |
| |
| <span class="c1"># The inferred schema can be visualized using the printSchema() method |
| </span><span class="n">peopleDF</span><span class="p">.</span><span class="n">printSchema</span><span class="p">()</span> |
| <span class="c1"># root |
| # |-- age: long (nullable = true) |
| # |-- name: string (nullable = true) |
| </span> |
| <span class="c1"># Creates a temporary view using the DataFrame |
| </span><span class="n">peopleDF</span><span class="p">.</span><span class="n">createOrReplaceTempView</span><span class="p">(</span><span class="s">"people"</span><span class="p">)</span> |
| |
| <span class="c1"># SQL statements can be run by using the sql methods provided by spark |
| </span><span class="n">teenagerNamesDF</span> <span class="o">=</span> <span class="n">spark</span><span class="p">.</span><span class="n">sql</span><span class="p">(</span><span class="s">"SELECT name FROM people WHERE age BETWEEN 13 AND 19"</span><span class="p">)</span> |
| <span class="n">teenagerNamesDF</span><span class="p">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +------+ |
| # | name| |
| # +------+ |
| # |Justin| |
| # +------+ |
| </span> |
| <span class="c1"># Alternatively, a DataFrame can be created for a JSON dataset represented by |
| # an RDD[String] storing one JSON object per string |
| </span><span class="n">jsonStrings</span> <span class="o">=</span> <span class="p">[</span><span class="s">'{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}'</span><span class="p">]</span> |
| <span class="n">otherPeopleRDD</span> <span class="o">=</span> <span class="n">sc</span><span class="p">.</span><span class="n">parallelize</span><span class="p">(</span><span class="n">jsonStrings</span><span class="p">)</span> |
| <span class="n">otherPeople</span> <span class="o">=</span> <span class="n">spark</span><span class="p">.</span><span class="n">read</span><span class="p">.</span><span class="n">json</span><span class="p">(</span><span class="n">otherPeopleRDD</span><span class="p">)</span> |
| <span class="n">otherPeople</span><span class="p">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +---------------+----+ |
| # | address|name| |
| # +---------------+----+ |
| # |[Columbus,Ohio]| Yin| |
| # +---------------+----+</span></code></pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/datasource.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="r"> |
| <p>Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. using |
| the <code class="language-plaintext highlighter-rouge">read.json()</code> function, which loads data from a directory of JSON files where each line of the |
| files is a JSON object.</p> |
| |
| <p>Note that the file that is offered as <em>a json file</em> is not a typical JSON file. Each |
| line must contain a separate, self-contained valid JSON object. For more information, please see |
| <a href="http://jsonlines.org/">JSON Lines text format, also called newline-delimited JSON</a>.</p> |
| |
| <p>For a regular multi-line JSON file, set a named parameter <code class="language-plaintext highlighter-rouge">multiLine</code> to <code class="language-plaintext highlighter-rouge">TRUE</code>.</p> |
| |
| <div class="highlight"><pre class="codehilite"><code><span class="c1"># A JSON dataset is pointed to by path.</span><span class="w"> |
| </span><span class="c1"># The path can be either a single text file or a directory storing text files.</span><span class="w"> |
| </span><span class="n">path</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="s2">"examples/src/main/resources/people.json"</span><span class="w"> |
| </span><span class="c1"># Create a DataFrame from the file(s) pointed to by path</span><span class="w"> |
| </span><span class="n">people</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">read.json</span><span class="p">(</span><span class="n">path</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="c1"># The inferred schema can be visualized using the printSchema() method.</span><span class="w"> |
| </span><span class="n">printSchema</span><span class="p">(</span><span class="n">people</span><span class="p">)</span><span class="w"> |
| </span><span class="c1">## root</span><span class="w"> |
| </span><span class="c1">## |-- age: long (nullable = true)</span><span class="w"> |
| </span><span class="c1">## |-- name: string (nullable = true)</span><span class="w"> |
| |
| </span><span class="c1"># Register this DataFrame as a table.</span><span class="w"> |
| </span><span class="n">createOrReplaceTempView</span><span class="p">(</span><span class="n">people</span><span class="p">,</span><span class="w"> </span><span class="s2">"people"</span><span class="p">)</span><span class="w"> |
| |
| </span><span class="c1"># SQL statements can be run by using the sql methods.</span><span class="w"> |
| </span><span class="n">teenagers</span><span class="w"> </span><span class="o"><-</span><span class="w"> </span><span class="n">sql</span><span class="p">(</span><span class="s2">"SELECT name FROM people WHERE age >= 13 AND age <= 19"</span><span class="p">)</span><span class="w"> |
| </span><span class="n">head</span><span class="p">(</span><span class="n">teenagers</span><span class="p">)</span><span class="w"> |
| </span><span class="c1">## name</span><span class="w"> |
| </span><span class="c1">## 1 Justin</span></code></pre></div> |
| <div><small>Find full example code at "examples/src/main/r/RSparkSQLExample.R" in the Spark repo.</small></div> |
| |
| </div> |
| |
| <div data-lang="SQL"> |
| |
| <figure class="highlight"><pre><code class="language-sql" data-lang="sql"><span class="k">CREATE</span> <span class="k">TEMPORARY</span> <span class="k">VIEW</span> <span class="n">jsonTable</span> |
| <span class="k">USING</span> <span class="n">org</span><span class="p">.</span><span class="n">apache</span><span class="p">.</span><span class="n">spark</span><span class="p">.</span><span class="k">sql</span><span class="p">.</span><span class="n">json</span> |
| <span class="k">OPTIONS</span> <span class="p">(</span> |
| <span class="n">path</span> <span class="nv">"examples/src/main/resources/people.json"</span> |
| <span class="p">)</span> |
| |
| <span class="k">SELECT</span> <span class="o">*</span> <span class="k">FROM</span> <span class="n">jsonTable</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h2 id="data-source-option">Data Source Option</h2> |
| |
| <p>Data source options of JSON can be set via:</p> |
| <ul> |
| <li>the <code class="language-plaintext highlighter-rouge">.option</code>/<code class="language-plaintext highlighter-rouge">.options</code> methods of |
| <ul> |
| <li><code class="language-plaintext highlighter-rouge">DataFrameReader</code></li> |
| <li><code class="language-plaintext highlighter-rouge">DataFrameWriter</code></li> |
| <li><code class="language-plaintext highlighter-rouge">DataStreamReader</code></li> |
| <li><code class="language-plaintext highlighter-rouge">DataStreamWriter</code></li> |
| </ul> |
| </li> |
| <li>the built-in functions below |
| <ul> |
| <li><code class="language-plaintext highlighter-rouge">from_json</code></li> |
| <li><code class="language-plaintext highlighter-rouge">to_json</code></li> |
| <li><code class="language-plaintext highlighter-rouge">schema_of_json</code></li> |
| </ul> |
| </li> |
| <li><code class="language-plaintext highlighter-rouge">OPTIONS</code> clause at <a href="sql-ref-syntax-ddl-create-table-datasource.html">CREATE TABLE USING DATA_SOURCE</a></li> |
| </ul> |
| |
| <table class="table"> |
| <tr><th><b>Property Name</b></th><th><b>Default</b></th><th><b>Meaning</b></th><th><b>Scope</b></th></tr> |
| <tr> |
| <!-- TODO(SPARK-35433): Add timeZone to Data Source Option for CSV, too. --> |
| <td><code>timeZone</code></td> |
| <td>(value of <code>spark.sql.session.timeZone</code> configuration)</td> |
| <td>Sets the string that indicates a time zone ID to be used to format timestamps in the JSON datasources or partition values. The following formats of <code>timeZone</code> are supported:<br /> |
| <ul> |
| <li>Region-based zone ID: It should have the form 'area/city', such as 'America/Los_Angeles'.</li> |
| <li>Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.</li> |
| </ul> |
| Other short names like 'CST' are not recommended to use because they can be ambiguous. |
| </td> |
| <td>read/write</td> |
| </tr> |
| <tr> |
| <td><code>primitivesAsString</code></td> |
| <td><code>false</code></td> |
| <td>Infers all primitive values as a string type.</td> |
| <td>read</td> |
| </tr> |
| <tr> |
| <td><code>prefersDecimal</code></td> |
| <td><code>false</code></td> |
| <td>Infers all floating-point values as a decimal type. If the values do not fit in decimal, then it infers them as doubles.</td> |
| <td>read</td> |
| </tr> |
| <tr> |
| <td><code>allowComments</code></td> |
| <td><code>false</code></td> |
| <td>Ignores Java/C++ style comment in JSON records.</td> |
| <td>read</td> |
| </tr> |
| <tr> |
| <td><code>allowUnquotedFieldNames</code></td> |
| <td><code>false</code></td> |
| <td>Allows unquoted JSON field names.</td> |
| <td>read</td> |
| </tr> |
| <tr> |
| <td><code>allowSingleQuotes</code></td> |
| <td><code>true</code></td> |
| <td>Allows single quotes in addition to double quotes.</td> |
| <td>read</td> |
| </tr> |
| <tr> |
| <td><code>allowNumericLeadingZero</code></td> |
| <td><code>false</code></td> |
| <td>Allows leading zeros in numbers (e.g. 00012).</td> |
| <td>read</td> |
| </tr> |
| <tr> |
| <td><code>allowBackslashEscapingAnyCharacter</code></td> |
| <td><code>false</code></td> |
| <td>Allows accepting quoting of all character using backslash quoting mechanism.</td> |
| <td>read</td> |
| </tr> |
| <tr> |
| <td><code>mode</code></td> |
| <td><code>PERMISSIVE</code></td> |
| <td>Allows a mode for dealing with corrupt records during parsing.<br /> |
| <ul> |
| <li><code>PERMISSIVE</code>: when it meets a corrupted record, puts the malformed string into a field configured by <code>columnNameOfCorruptRecord</code>, and sets malformed fields to <code>null</code>. To keep corrupt records, an user can set a string type field named <code>columnNameOfCorruptRecord</code> in an user-defined schema. If a schema does not have the field, it drops corrupt records during parsing. When inferring a schema, it implicitly adds a <code>columnNameOfCorruptRecord</code> field in an output schema.</li> |
| <li><code>DROPMALFORMED</code>: ignores the whole corrupted records. This mode is unsupported in the JSON built-in functions.</li> |
| <li><code>FAILFAST</code>: throws an exception when it meets corrupted records.</li> |
| </ul> |
| </td> |
| <td>read</td> |
| </tr> |
| <tr> |
| <td><code>columnNameOfCorruptRecord</code></td> |
| <td>(value of <code>spark.sql.columnNameOfCorruptRecord</code> configuration)</td> |
| <td>Allows renaming the new field having malformed string created by <code>PERMISSIVE</code> mode. This overrides spark.sql.columnNameOfCorruptRecord.</td> |
| <td>read</td> |
| </tr> |
| <tr> |
| <td><code>dateFormat</code></td> |
| <td><code>yyyy-MM-dd</code></td> |
| <td>Sets the string that indicates a date format. Custom date formats follow the formats at <a href="https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html"> datetime pattern</a>. This applies to date type.</td> |
| <td>read/write</td> |
| </tr> |
| <tr> |
| <td><code>timestampFormat</code></td> |
| <td><code>yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]</code></td> |
| <td>Sets the string that indicates a timestamp format. Custom date formats follow the formats at <a href="https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html"> datetime pattern</a>. This applies to timestamp type.</td> |
| <td>read/write</td> |
| </tr> |
| <tr> |
| <td><code>timestampNTZFormat</code></td> |
| <td>yyyy-MM-dd'T'HH:mm:ss[.SSS]</td> |
| <td>Sets the string that indicates a timestamp without timezone format. Custom date formats follow the formats at <a href="https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html">Datetime Patterns</a>. This applies to timestamp without timezone type, note that zone-offset and time-zone components are not supported when writing or reading this data type.</td> |
| <td>read/write</td> |
| </tr> |
| <tr> |
| <td><code>multiLine</code></td> |
| <td><code>false</code></td> |
| <td>Parse one record, which may span multiple lines, per file. JSON built-in functions ignore this option.</td> |
| <td>read</td> |
| </tr> |
| <tr> |
| <td><code>allowUnquotedControlChars</code></td> |
| <td><code>false</code></td> |
| <td>Allows JSON Strings to contain unquoted control characters (ASCII characters with value less than 32, including tab and line feed characters) or not.</td> |
| <td>read</td> |
| </tr> |
| <tr> |
| <td><code>encoding</code></td> |
| <td>Detected automatically when <code>multiLine</code> is set to <code>true</code> (for reading), <code>UTF-8</code> (for writing)</td> |
| <td>For reading, allows to forcibly set one of standard basic or extended encoding for the JSON files. For example UTF-16BE, UTF-32LE. For writing, Specifies encoding (charset) of saved json files. JSON built-in functions ignore this option.</td> |
| <td>read/write</td> |
| </tr> |
| <tr> |
| <td><code>lineSep</code></td> |
| <td><code>\r</code>, <code>\r\n</code>, <code>\n</code> (for reading), <code>\n</code> (for writing)</td> |
| <td>Defines the line separator that should be used for parsing. JSON built-in functions ignore this option.</td> |
| <td>read/write</td> |
| </tr> |
| <tr> |
| <td><code>samplingRatio</code></td> |
| <td><code>1.0</code></td> |
| <td>Defines fraction of input JSON objects used for schema inferring.</td> |
| <td>read</td> |
| </tr> |
| <tr> |
| <td><code>dropFieldIfAllNull</code></td> |
| <td><code>false</code></td> |
| <td>Whether to ignore column of all null values or empty array/struct during schema inference.</td> |
| <td>read</td> |
| </tr> |
| <tr> |
| <td><code>locale</code></td> |
| <td><code>en-US</code></td> |
| <td>Sets a locale as language tag in IETF BCP 47 format. For instance, <code>locale</code> is used while parsing dates and timestamps.</td> |
| <td>read</td> |
| </tr> |
| <tr> |
| <td><code>allowNonNumericNumbers</code></td> |
| <td><code>true</code></td> |
| <td>Allows JSON parser to recognize set of “Not-a-Number” (NaN) tokens as legal floating number values.<br /> |
| <ul> |
| <li><code>+INF</code>: for positive infinity, as well as alias of <code>+Infinity</code> and <code>Infinity</code>.</li> |
| <li><code>-INF</code>: for negative infinity, alias <code>-Infinity</code>.</li> |
| <li><code>NaN</code>: for other not-a-numbers, like result of division by zero.</li> |
| </ul> |
| </td> |
| <td>read</td> |
| </tr> |
| <tr> |
| <td><code>compression</code></td> |
| <td>(none)</td> |
| <td>Compression codec to use when saving to file. This can be one of the known case-insensitive shorten names (none, bzip2, gzip, lz4, snappy and deflate). JSON built-in functions ignore this option.</td> |
| <td>write</td> |
| </tr> |
| <tr> |
| <td><code>ignoreNullFields</code></td> |
| <td>(value of <code>spark.sql.jsonGenerator.ignoreNullFields</code> configuration)</td> |
| <td>Whether to ignore null fields when generating JSON objects.</td> |
| <td>write</td> |
| </tr> |
| </table> |
| <p>Other generic options can be found in <a href="https://spark.apache.org/docs/latest/sql-data-sources-generic-options.html"> Generic File Source Options</a>.</p> |
| |
| |
| </div> |
| |
| <!-- /container --> |
| </div> |
| |
| <script src="js/vendor/jquery-3.5.1.min.js"></script> |
| <script src="js/vendor/bootstrap.bundle.min.js"></script> |
| <script src="js/vendor/anchor.min.js"></script> |
| <script src="js/main.js"></script> |
| <script type="text/javascript" src="https://cdn.jsdelivr.net/npm/docsearch.js@2/dist/cdn/docsearch.min.js"></script> |
| <script type="text/javascript"> |
| // DocSearch is entirely free and automated. DocSearch is built in two parts: |
| // 1. a crawler which we run on our own infrastructure every 24 hours. It follows every link |
| // in your website and extract content from every page it traverses. It then pushes this |
| // content to an Algolia index. |
| // 2. a JavaScript snippet to be inserted in your website that will bind this Algolia index |
| // to your search input and display its results in a dropdown UI. If you want to find more |
| // details on how works DocSearch, check the docs of DocSearch. |
| docsearch({ |
| apiKey: 'd62f962a82bc9abb53471cb7b89da35e', |
| appId: 'RAI69RXRSK', |
| indexName: 'apache_spark', |
| inputSelector: '#docsearch-input', |
| enhancedSearchInput: true, |
| algoliaOptions: { |
| 'facetFilters': ["version:3.3.4"] |
| }, |
| debug: false // Set debug to true if you want to inspect the dropdown |
| }); |
| |
| </script> |
| |
| <!-- MathJax Section --> |
| <script type="text/x-mathjax-config"> |
| MathJax.Hub.Config({ |
| TeX: { equationNumbers: { autoNumber: "AMS" } } |
| }); |
| </script> |
| <script> |
| // Note that we load MathJax this way to work with local file (file://), HTTP and HTTPS. |
| // We could use "//cdn.mathjax...", but that won't support "file://". |
| (function(d, script) { |
| script = d.createElement('script'); |
| script.type = 'text/javascript'; |
| script.async = true; |
| script.onload = function(){ |
| MathJax.Hub.Config({ |
| tex2jax: { |
| inlineMath: [ ["$", "$"], ["\\\\(","\\\\)"] ], |
| displayMath: [ ["$$","$$"], ["\\[", "\\]"] ], |
| processEscapes: true, |
| skipTags: ['script', 'noscript', 'style', 'textarea', 'pre'] |
| } |
| }); |
| }; |
| script.src = ('https:' == document.location.protocol ? 'https://' : 'http://') + |
| 'cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.1/MathJax.js' + |
| '?config=TeX-AMS-MML_HTMLorMML'; |
| d.getElementsByTagName('head')[0].appendChild(script); |
| }(document)); |
| </script> |
| </body> |
| </html> |