blob: 279eba6c066713193c436698dedfc924d85cfdbc [file] [log] [blame]
<!doctype html><html lang=en class=no-js><head><meta charset=utf-8><meta http-equiv=x-ua-compatible content="IE=edge"><meta name=viewport content="width=device-width,initial-scale=1"><title>Beam SQL extension: CREATE EXTERNAL TABLE Statement</title><meta name=description content="Apache Beam is an open source, unified model and set of language-specific SDKs for defining and executing data processing workflows, and also data ingestion and integration flows, supporting Enterprise Integration Patterns (EIPs) and Domain Specific Languages (DSLs). Dataflow pipelines simplify the mechanics of large-scale batch and streaming data processing and can run on a number of runtimes like Apache Flink, Apache Spark, and Google Cloud Dataflow (a cloud service). Beam also brings DSL in different languages, allowing users to easily implement their data integration processes."><link href="https://fonts.googleapis.com/css?family=Roboto:100,300,400,500,700" rel=stylesheet><link rel=preload href=/scss/main.min.408fddfe3e8a45f87a5a8c9a839d77db667c1c534e5e5cd0d957ffc3dd6c14cf.css as=style><link href=/scss/main.min.408fddfe3e8a45f87a5a8c9a839d77db667c1c534e5e5cd0d957ffc3dd6c14cf.css rel=stylesheet integrity><script src=https://code.jquery.com/jquery-2.2.4.min.js></script><style>.body__contained img{max-width:100%}</style><script type=text/javascript src=/js/bootstrap.min.2979f9a6e32fc42c3e7406339ee9fe76b31d1b52059776a02b4a7fa6a4fd280a.js defer></script>
<script type=text/javascript src=/js/language-switch-v2.min.121952b7980b920320ab229551857669209945e39b05ba2b433a565385ca44c6.js defer></script>
<script type=text/javascript src=/js/fix-menu.min.039174b67107465f2090a493f91e126f7aa797f29420f9edab8a54d9dd4b3d2d.js defer></script>
<script type=text/javascript src=/js/section-nav.min.1405fd5e70fab5f6c54037c269b1d137487d8f3d1b3009032525f6db3fbce991.js defer></script>
<script type=text/javascript src=/js/page-nav.min.af231204c9c52c5089d53a4c02739eacbb7f939e3be1c6ffcc212e0ac4dbf879.js defer></script>
<script type=text/javascript src=/js/expandable-list.min.75a4526624a3b8898fe7fb9e3428c205b581f8b38c7926922467aef17eac69f2.js defer></script>
<script type=text/javascript src=/js/copy-to-clipboard.min.364c06423d7e8993fc42bb4abc38c03195bc8386db26d18774ce775d08d5b18d.js defer></script>
<script type=text/javascript src=/js/calendar.min.336664054fa0f52b08bbd4e3c59b5cb6d63dcfb2b4d602839746516b0817446b.js defer></script>
<script type=text/javascript src=/js/fix-playground-nested-scroll.min.0283f1037cb1b9d5074c6eaf041292b524a8148a7cdb803d5ccd6d1fc4eb3253.js defer></script>
<script type=text/javascript src=/js/anchor-content-jump-fix.min.22d3240f81632e4c11179b9d2aaf37a40da9414333c43aa97344e8b21a7df0e4.js defer></script>
<link rel=alternate type=application/rss+xml title="Apache Beam" href=/feed.xml><link rel=canonical href=/documentation/dsls/sql/extensions/create-external-table/ data-proofer-ignore><link rel="shortcut icon" type=image/x-icon href=/images/favicon.ico><link rel=stylesheet href=https://use.fontawesome.com/releases/v5.4.1/css/all.css integrity=sha384-5sAR7xN1Nv6T6+dT2mhtzEpVJvfS3NScPQTrOxhwjIuvcA67KV2R5Jz6kr4abQsz crossorigin=anonymous><link rel=stylesheet href=https://unpkg.com/swiper@8/swiper-bundle.min.css><script async src=https://platform.twitter.com/widgets.js></script>
<script>(function(e,t,n,s,o,i,a){e.GoogleAnalyticsObject=o,e[o]=e[o]||function(){(e[o].q=e[o].q||[]).push(arguments)},e[o].l=1*new Date,i=t.createElement(n),a=t.getElementsByTagName(n)[0],i.async=1,i.src=s,a.parentNode.insertBefore(i,a)})(window,document,"script","//www.google-analytics.com/analytics.js","ga"),ga("create","UA-73650088-1","auto"),ga("send","pageview")</script><script>(function(e,t,n,s,o,i){e.hj=e.hj||function(){(e.hj.q=e.hj.q||[]).push(arguments)},e._hjSettings={hjid:2182187,hjsv:6},o=t.getElementsByTagName("head")[0],i=t.createElement("script"),i.async=1,i.src=n+e._hjSettings.hjid+s+e._hjSettings.hjsv,o.appendChild(i)})(window,document,"https://static.hotjar.com/c/hotjar-",".js?sv=")</script></head><body class=body data-spy=scroll data-target=.page-nav data-offset=0><nav class="navigation-bar-mobile header navbar navbar-fixed-top"><div class=navbar-header><a href=/ class=navbar-brand><img alt=Brand style=height:46px;width:43px src=/images/beam_logo_navbar_mobile.png></a>
<a class=navbar-link href=/get-started/>Get Started</a>
<a class=navbar-link href=/documentation/>Documentation</a>
<button type=button class="navbar-toggle menu-open" aria-expanded=false aria-controls=navbar onclick=openMenu()>
<span class=sr-only>Toggle navigation</span>
<span class=icon-bar></span>
<span class=icon-bar></span>
<span class=icon-bar></span></button></div><div class="navbar-mask closed"></div><div id=navbar class="navbar-container closed"><button type=button class=navbar-toggle aria-expanded=false aria-controls=navbar id=closeMenu>
<span class=sr-only>Toggle navigation</span>
<span class=icon-bar></span>
<span class=icon-bar></span>
<span class=icon-bar></span></button><ul class="nav navbar-nav"><li><div class=searchBar-mobile><script>(function(){var t,n="012923275103528129024:4emlchv9wzi",e=document.createElement("script");e.type="text/javascript",e.async=!0,e.src="https://cse.google.com/cse.js?cx="+n,t=document.getElementsByTagName("script")[0],t.parentNode.insertBefore(e,t)})()</script><gcse:search></gcse:search></div></li><li><a class=navbar-link href=/about>About</a></li><li><a class=navbar-link href=/get-started/>Get Started</a></li><li><span class=navbar-link>Documentation</span><ul><li><a href=/documentation/>General</a></li><li><a href=/documentation/sdks/java/>Languages</a></li><li><a href=/documentation/runners/capability-matrix/>Runners</a></li><li><a href=/documentation/io/connectors/>I/O Connectors</a></li></ul></li><li><a class=navbar-link href=/roadmap/>Roadmap</a></li><li><a class=navbar-link href=/community/>Community</a></li><li><a class=navbar-link href=/contribute/>Contribute</a></li><li><a class=navbar-link href=/blog/>Blog</a></li><li><a class=navbar-link href=/case-studies/>Case Studies</a></li></ul><ul class="nav navbar-nav navbar-right"><li><a href=https://github.com/apache/beam/edit/master/website/www/site/content/en/documentation/dsls/sql/extensions/create-external-table.md data-proofer-ignore><svg xmlns="http://www.w3.org/2000/svg" width="25" height="24" fill="none" viewBox="0 0 25 24"><path stroke="#ff6d00" stroke-linecap="round" stroke-linejoin="round" stroke-width="2.75" d="M4.543 20h4l10.5-10.5c.53-.53.828-1.25.828-2s-.298-1.47-.828-2-1.25-.828-2-.828-1.47.298-2 .828L4.543 16v4zm9.5-13.5 4 4"/></svg></a></li><li class=dropdown><a href=# class=dropdown-toggle id=apache-dropdown data-toggle=dropdown role=button aria-haspopup=true aria-expanded=false><img src=https://www.apache.org/foundation/press/kit/feather_small.png alt="Apache Logo" style=height:20px>
&nbsp;Apache
<span class=arrow-icon><svg xmlns="http://www.w3.org/2000/svg" width="20" height="20" fill="none" viewBox="0 0 20 20"><circle cx="10" cy="10" r="10" fill="#ff6d00"/><path stroke="#fff" stroke-linecap="round" stroke-linejoin="round" stroke-width="2" d="M8.535 5.28l4.573 4.818-4.573 4.403"/></svg></span></a><ul class="dropdown-menu dropdown-menu-right"><li><a target=_blank href=https://www.apache.org/>ASF Homepage</a></li><li><a target=_blank href=https://www.apache.org/licenses/>License</a></li><li><a target=_blank href=https://www.apache.org/security/>Security</a></li><li><a target=_blank href=https://www.apache.org/foundation/thanks.html>Thanks</a></li><li><a target=_blank href=https://www.apache.org/foundation/sponsorship.html>Sponsorship</a></li><li><a target=_blank href=https://www.apache.org/foundation/policies/conduct>Code of Conduct</a></li></ul></li></ul></div></nav><nav class=navigation-bar-desktop><a href=/ class=navbar-logo><img src=/images/beam_logo_navbar.png alt="Beam Logo"></a><div class=navbar-bar-left><div class=navbar-links><a class=navbar-link href=/about>About</a>
<a class=navbar-link href=/get-started/>Get Started</a><li class="dropdown navbar-dropdown navbar-dropdown-documentation"><a href=# class="dropdown-toggle navbar-link" role=button aria-haspopup=true aria-expanded=false>Documentation
<span><svg xmlns="http://www.w3.org/2000/svg" width="12" height="11" fill="none" viewBox="0 0 12 11"><path stroke="#ff6d00" stroke-linecap="round" stroke-linejoin="round" stroke-width="2" d="M10.666 4.535 5.847 9.108 1.444 4.535"/></svg></span></a><ul class=dropdown-menu><li><a class=navbar-dropdown-menu-link href=/documentation/>General</a></li><li><a class=navbar-dropdown-menu-link href=/documentation/sdks/java/>Languages</a></li><li><a class=navbar-dropdown-menu-link href=/documentation/runners/capability-matrix/>Runners</a></li><li><a class=navbar-dropdown-menu-link href=/documentation/io/connectors/>I/O Connectors</a></li></ul></li><a class=navbar-link href=/roadmap/>Roadmap</a>
<a class=navbar-link href=/community/>Community</a>
<a class=navbar-link href=/contribute/>Contribute</a>
<a class=navbar-link href=/blog/>Blog</a>
<a class=navbar-link href=/case-studies/>Case Studies</a></div><div id=iconsBar><a type=button onclick=showSearch()><svg xmlns="http://www.w3.org/2000/svg" width="25" height="24" fill="none" viewBox="0 0 25 24"><path stroke="#ff6d00" stroke-linecap="round" stroke-linejoin="round" stroke-width="2.75" d="M10.191 17c3.866.0 7-3.134 7-7s-3.134-7-7-7-7 3.134-7 7 3.134 7 7 7zm11 4-6-6"/></svg></a><a target=_blank href=https://github.com/apache/beam/edit/master/website/www/site/content/en/documentation/dsls/sql/extensions/create-external-table.md data-proofer-ignore><svg xmlns="http://www.w3.org/2000/svg" width="25" height="24" fill="none" viewBox="0 0 25 24"><path stroke="#ff6d00" stroke-linecap="round" stroke-linejoin="round" stroke-width="2.75" d="M4.543 20h4l10.5-10.5c.53-.53.828-1.25.828-2s-.298-1.47-.828-2-1.25-.828-2-.828-1.47.298-2 .828L4.543 16v4zm9.5-13.5 4 4"/></svg></a><li class="dropdown navbar-dropdown navbar-dropdown-apache"><a href=# class=dropdown-toggle role=button aria-haspopup=true aria-expanded=false><img src=https://www.apache.org/foundation/press/kit/feather_small.png alt="Apache Logo" style=height:20px>
&nbsp;Apache
<span class=arrow-icon><svg xmlns="http://www.w3.org/2000/svg" width="20" height="20" fill="none" viewBox="0 0 20 20"><circle cx="10" cy="10" r="10" fill="#ff6d00"/><path stroke="#fff" stroke-linecap="round" stroke-linejoin="round" stroke-width="2" d="M8.535 5.28l4.573 4.818-4.573 4.403"/></svg></span></a><ul class=dropdown-menu><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/>ASF Homepage</a></li><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/licenses/>License</a></li><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/security/>Security</a></li><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/foundation/thanks.html>Thanks</a></li><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/foundation/sponsorship.html>Sponsorship</a></li><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/foundation/policies/conduct>Code of Conduct</a></li></ul></li></div><div class="searchBar disappear"><script>(function(){var t,n="012923275103528129024:4emlchv9wzi",e=document.createElement("script");e.type="text/javascript",e.async=!0,e.src="https://cse.google.com/cse.js?cx="+n,t=document.getElementsByTagName("script")[0],t.parentNode.insertBefore(e,t)})()</script><gcse:search></gcse:search>
<a type=button onclick=endSearch()><svg xmlns="http://www.w3.org/2000/svg" width="25" height="25" fill="none" viewBox="0 0 25 25"><path stroke="#ff6d00" stroke-linecap="round" stroke-linejoin="round" stroke-width="2.75" d="M21.122 20.827 4.727 4.432M21.122 4.43 4.727 20.827"/></svg></a></div></div></nav><div class=header-push></div><div class="top-banners swiper"><div class=swiper-wrapper><div class=swiper-slide><a href=https://tour.beam.apache.org><img class=banner-img-desktop src=/images/banners/tour-of-beam/tour-of-beam-desktop.png alt="Start Tour of Beam">
<img class=banner-img-mobile src=/images/banners/tour-of-beam/tour-of-beam-mobile.png alt="Start Tour of Beam"></a></div><div class=swiper-slide><a href=https://beam.apache.org/documentation/ml/overview/><img class=banner-img-desktop src=/images/banners/machine-learning/machine-learning-desktop.jpg alt="Machine Learning">
<img class=banner-img-mobile src=/images/banners/machine-learning/machine-learning-mobile.jpg alt="Machine Learning"></a></div></div><div class=swiper-pagination></div><div class=swiper-button-prev></div><div class=swiper-button-next></div></div><script src=/js/swiper-bundle.min.min.e0e8f81b0b15728d35ff73c07f42ddbb17a108d6f23df4953cb3e60df7ade675.js></script>
<script src=/js/sliders/top-banners.min.afa7d0a19acf7a3b28ca369490b3d401a619562a2a4c9612577be2f66a4b9855.js></script>
<script>function showSearch(){addPlaceholder();var e,t=document.querySelector(".searchBar");t.classList.remove("disappear"),e=document.querySelector("#iconsBar"),e.classList.add("disappear")}function addPlaceholder(){$("input:text").attr("placeholder","What are you looking for?")}function endSearch(){var e,t=document.querySelector(".searchBar");t.classList.add("disappear"),e=document.querySelector("#iconsBar"),e.classList.remove("disappear")}function blockScroll(){$("body").toggleClass("fixedPosition")}function openMenu(){addPlaceholder(),blockScroll()}</script><div class="clearfix container-main-content"><div class="section-nav closed" data-offset-top=90 data-offset-bottom=500><span class="section-nav-back glyphicon glyphicon-menu-left"></span><nav><ul class=section-nav-list data-section-nav><li><span class=section-nav-list-main-title>Languages</span></li><li><span class=section-nav-list-title>Java</span><ul class=section-nav-list><li><a href=/documentation/sdks/java/>Java SDK overview</a></li><li><a href=https://beam.apache.org/releases/javadoc/2.56.0/ target=_blank>Java SDK API reference <img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></li><li><a href=/documentation/sdks/java-dependencies/>Java SDK dependencies</a></li><li><a href=/documentation/sdks/java-extensions/>Java SDK extensions</a></li><li><a href=/documentation/sdks/java-thirdparty/>Java 3rd party extensions</a></li><li><a href=/documentation/sdks/java/testing/nexmark/>Nexmark benchmark suite</a></li><li><a href=/documentation/sdks/java/testing/tpcds/>TPC-DS benchmark suite</a></li><li><a href=/documentation/sdks/java-multi-language-pipelines/>Java multi-language pipelines quickstart</a></li></ul></li><li><span class=section-nav-list-title>Python</span><ul class=section-nav-list><li><a href=/documentation/sdks/python/>Python SDK overview</a></li><li><a href=https://beam.apache.org/releases/pydoc/2.56.0/ target=_blank>Python SDK API reference <img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></li><li><a href=/documentation/sdks/python-dependencies/>Python SDK dependencies</a></li><li><a href=/documentation/sdks/python-streaming/>Python streaming pipelines</a></li><li><a href=/documentation/sdks/python-type-safety/>Ensuring Python type safety</a></li><li><a href=/documentation/sdks/python-machine-learning/>Machine Learning</a></li><li><a href=/documentation/sdks/python-pipeline-dependencies/>Managing pipeline dependencies</a></li><li><a href=/documentation/sdks/python-multi-language-pipelines/>Python multi-language pipelines quickstart</a></li><li><a href=/documentation/sdks/python-unrecoverable-errors/>Python Unrecoverable Errors</a></li></ul></li><li><span class=section-nav-list-title>Go</span><ul class=section-nav-list><li><a href=/documentation/sdks/go/>Go SDK overview</a></li><li><a href=https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam target=_blank>Go SDK API reference <img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a><li><a href=/documentation/sdks/go-dependencies/>Go SDK dependencies</a></li><li><a href=/documentation/sdks/go-cross-compilation/>Cross compilation</a></li></li></ul></li><li><span class=section-nav-list-title>Typescript</span><ul class=section-nav-list><li><a href=/documentation/sdks/typescript/>Typescript SDK overview</a></li><li><a href=https://beam.apache.org/releases/typedoc/current/ target=_blank>Typescript SDK API reference <img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></li></ul></li><li><span class=section-nav-list-title>Scala</span><ul class=section-nav-list><li><a href=/documentation/sdks/scala/>Scio</a></li><li><a href=https://spotify.github.io/scio/api/com/spotify/scio/index.html target=_blank>Scio SDK API reference <img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></li></ul></li><li><span class=section-nav-list-title>Yaml</span><ul class=section-nav-list><li><a href=/documentation/sdks/yaml/>Yaml overview</a></li><li><a href=/documentation/sdks/yaml-udf/>Yaml User Defined Functions</a></li><li><a href=/documentation/sdks/yaml-combine/>Yaml Aggregation</a></li><li><a href=/documentation/sdks/yaml-errors/>Error handling</a></li><li><a href=/documentation/sdks/yaml-inline-python/>Inlining Python</a></li><li><a href=https://beam.apache.org/releases/yamldoc/current/ target=_blank>YAML API reference <img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></ul></li><li><span class=section-nav-list-title>SQL</span><ul class=section-nav-list><li><a href=/documentation/dsls/sql/overview/>Overview</a></li><li><a href=/documentation/dsls/sql/walkthrough/>Walkthrough</a></li><li><a href=/documentation/dsls/sql/shell/>Shell</a></li><li class=section-nav-item--collapsible><span class=section-nav-list-title>Apache Calcite dialect</span><ul class=section-nav-list><li><a href=/documentation/dsls/sql/calcite/overview/>Calcite support overview</a></li><li><a href=/documentation/dsls/sql/calcite/query-syntax/>Query syntax</a></li><li><a href=/documentation/dsls/sql/calcite/lexical/>Lexical structure</a></li><li><a href=/documentation/dsls/sql/calcite/data-types/>Data types</a></li><li><a href=/documentation/dsls/sql/calcite/scalar-functions/>Scalar functions</a></li><li><a href=/documentation/dsls/sql/calcite/aggregate-functions/>Aggregate functions</a></li></ul></li><li class=section-nav-item--collapsible><span class=section-nav-list-title>ZetaSQL dialect</span><ul class=section-nav-list><li><a href=/documentation/dsls/sql/zetasql/overview/>ZetaSQL support overview</a></li><li><a href=/documentation/dsls/sql/zetasql/syntax/>Function call rules</a></li><li><a href=/documentation/dsls/sql/zetasql/conversion-rules/>Conversion rules</a></li><li><a href=/documentation/dsls/sql/zetasql/query-syntax/>Query syntax</a></li><li><a href=/documentation/dsls/sql/zetasql/lexical/>Lexical structure</a></li><li><a href=/documentation/dsls/sql/zetasql/data-types/>Data types</a></li><li><a href=/documentation/dsls/sql/zetasql/operators/>Operators</a></li><li class=section-nav-item--collapsible><span class=section-nav-list-title>Scalar functions</span><ul class=section-nav-list><li><a href=/documentation/dsls/sql/zetasql/string-functions/>String functions</a></li><li><a href=/documentation/dsls/sql/zetasql/math-functions/>Mathematical functions</a></li><li><a href=/documentation/dsls/sql/zetasql/conditional-expressions/>Conditional expressions</a></li></ul></li><li><a href=/documentation/dsls/sql/zetasql/aggregate-functions/>Aggregate functions</a></li></ul></li><li class=section-nav-item--collapsible><span class=section-nav-list-title>Beam SQL extensions</span><ul class=section-nav-list><li><a href=/documentation/dsls/sql/extensions/create-external-table/>CREATE EXTERNAL TABLE</a></li><li><a href=/documentation/dsls/sql/extensions/windowing-and-triggering/>Windowing & triggering</a></li><li><a href=/documentation/dsls/sql/extensions/joins/>Joins</a></li><li><a href=/documentation/dsls/sql/extensions/user-defined-functions/>User-defined functions</a></li><li><a href=/documentation/dsls/sql/extensions/set/>SET pipeline options</a></li></ul></li></ul></li><li><span class=section-nav-list-title>DataFrames</span><ul class=section-nav-list><li><a href=/documentation/dsls/dataframes/overview/>Overview</a></li><li><a href=/documentation/dsls/dataframes/differences-from-pandas/>Differences from pandas</a></li><li><a href=https://github.com/apache/beam/tree/master/sdks/python/apache_beam/examples/dataframe target=_blank>Example pipelines <img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></li><li><a href=https://beam.apache.org/releases/pydoc/2.56.0/apache_beam.dataframe.html target=_blank>DataFrame API reference <img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></li></ul></li></ul></nav></div><nav class="page-nav clearfix" data-offset-top=90 data-offset-bottom=500><nav id=TableOfContents><ul><li><a href=#syntax>Syntax</a></li><li><a href=#bigquery>BigQuery</a><ul><li><a href=#syntax-1>Syntax</a></li><li><a href=#read-mode>Read Mode</a></li><li><a href=#write-mode>Write Mode</a></li><li><a href=#schema>Schema</a></li><li><a href=#example>Example</a></li></ul></li><li><a href=#cloud-bigtable>Cloud Bigtable</a><ul><li><a href=#syntax-2>Syntax</a></li><li><a href=#read-mode-1>Read Mode</a></li><li><a href=#write-mode-1>Write Mode</a></li><li><a href=#example-1>Example</a></li></ul></li><li><a href=#pubsub>Pub/Sub</a><ul><li><a href=#syntax-3>Syntax</a><ul><li><a href=#nested-mode>Nested mode</a></li><li><a href=#flattened-mode>Flattened mode</a></li></ul></li><li><a href=#read-mode-2>Read Mode</a></li><li><a href=#write-mode-2>Write Mode</a></li><li><a href=#schema-1>Schema</a></li><li><a href=#supported-payload>Supported Payload</a></li><li><a href=#example-2>Example</a></li></ul></li><li><a href=#pubsub-lite>Pub/Sub Lite</a><ul><li><a href=#syntax-4>Syntax</a></li><li><a href=#read-mode-3>Read Mode</a></li><li><a href=#write-mode-3>Write Mode</a></li><li><a href=#supported-payload-1>Supported Payload</a></li><li><a href=#example-3>Example</a></li></ul></li><li><a href=#kafka>Kafka</a><ul><li><a href=#syntax-5>Syntax</a><ul><li><a href=#flattened-mode-1>Flattened mode</a></li><li><a href=#nested-mode-1>Nested mode</a></li></ul></li><li><a href=#read-mode-4>Read Mode</a></li><li><a href=#write-mode-4>Write Mode</a></li><li><a href=#supported-formats>Supported Formats</a></li><li><a href=#schema-2>Schema</a></li></ul></li><li><a href=#mongodb>MongoDB</a><ul><li><a href=#syntax-6>Syntax</a></li><li><a href=#read-mode-5>Read Mode</a></li><li><a href=#write-mode-5>Write Mode</a></li><li><a href=#schema-3>Schema</a></li><li><a href=#example-4>Example</a></li></ul></li><li><a href=#text>Text</a><ul><li><a href=#syntax-7>Syntax</a></li><li><a href=#read-mode-6>Read Mode</a></li><li><a href=#write-mode-6>Write Mode</a></li><li><a href=#supported-payload-2>Supported Payload</a></li><li><a href=#schema-4>Schema</a></li><li><a href=#example-5>Example</a></li></ul></li><li><a href=#generic-payload-handling>Generic Payload Handling</a></li><li><a href=#generic-dlq-handling>Generic DLQ Handling</a></li></ul></nav></nav><div class="body__contained body__section-nav"><h1 id=beam-sql-extensions-create-external-table>Beam SQL extensions: CREATE EXTERNAL TABLE</h1><p>Beam SQL&rsquo;s <code>CREATE EXTERNAL TABLE</code> statement registers a virtual table that maps to an
<a href=/documentation/io/built-in/>external storage system</a>.
For some storage systems, <code>CREATE EXTERNAL TABLE</code> does not create a physical table until
a write occurs. After the physical table exists, you can access the table with
the <code>SELECT</code>, <code>JOIN</code>, and <code>INSERT INTO</code> statements.</p><p>The <code>CREATE EXTERNAL TABLE</code> statement includes a schema and extended clauses.</p><h2 id=syntax>Syntax</h2><pre tabindex=0><code>CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName (tableElement [, tableElement ]*)
TYPE type
[LOCATION location]
[TBLPROPERTIES tblProperties]
simpleType: TINYINT | SMALLINT | INTEGER | BIGINT | FLOAT | DOUBLE | DECIMAL | BOOLEAN | DATE | TIME | TIMESTAMP | CHAR | VARCHAR
fieldType: simpleType | MAP&lt;simpleType, fieldType&gt; | ARRAY&lt;fieldType&gt; | ROW&lt;tableElement [, tableElement ]*&gt;
tableElement: columnName fieldType [ NOT NULL ]
</code></pre><ul><li><code>IF NOT EXISTS</code>: Optional. If the table is already registered, Beam SQL
ignores the statement instead of returning an error.</li><li><code>tableName</code>: The case sensitive name of the table to create and register,
specified as an
<a href=/documentation/dsls/sql/calcite/lexical#identifiers>Identifier</a>.
The table name does not need to match the name in the underlying data
storage system.</li><li><code>tableElement</code>: <code>columnName</code> <code>fieldType</code> <code>[ NOT NULL ]</code><ul><li><code>columnName</code>: The case sensitive name of the column, specified as a
backtick_quoted_expression.</li><li><code>fieldType</code>: The field&rsquo;s type, specified as one of the following types:<ul><li><code>simpleType</code>: <code>TINYINT</code>, <code>SMALLINT</code>, <code>INTEGER</code>, <code>BIGINT</code>, <code>FLOAT</code>,
<code>DOUBLE</code>, <code>DECIMAL</code>, <code>BOOLEAN</code>, <code>DATE</code>, <code>TIME</code>, <code>TIMESTAMP</code>, <code>CHAR</code>,
<code>VARCHAR</code></li><li><code>MAP&lt;simpleType, fieldType></code></li><li><code>ARRAY&lt;fieldType></code></li><li><code>ROW&lt;tableElement [, tableElement ]*></code></li></ul></li><li><code>NOT NULL</code>: Optional. Indicates that the column is not nullable.</li></ul></li><li><code>type</code>: The I/O transform that backs the virtual table, specified as an
<a href=/documentation/dsls/sql/calcite/lexical/#identifiers>Identifier</a>
with one of the following values:<ul><li><code>bigquery</code></li><li><code>bigtable</code></li><li><code>pubsub</code></li><li><code>kafka</code></li><li><code>text</code></li></ul></li><li><code>location</code>: The I/O specific location of the underlying table, specified as
a <a href=/documentation/dsls/sql/calcite/lexical/#string-literals>String
Literal</a>.
See the I/O specific sections for <code>location</code> format requirements.</li><li><code>tblProperties</code>: The I/O specific quoted key value JSON object with extra
configuration, specified as a <a href=/documentation/dsls/sql/calcite/lexical/#string-literals>String
Literal</a>.
See the I/O specific sections for <code>tblProperties</code> format requirements.</li></ul><h2 id=bigquery>BigQuery</h2><h3 id=syntax-1>Syntax</h3><pre tabindex=0><code>CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName (tableElement [, tableElement ]*)
TYPE bigquery
LOCATION &#39;[PROJECT_ID]:[DATASET].[TABLE]&#39;
TBLPROPERTIES &#39;{&#34;method&#34;: &#34;DIRECT_READ&#34;}&#39;
</code></pre><ul><li><code>LOCATION</code>: Location of the table in the BigQuery CLI format.<ul><li><code>PROJECT_ID</code>: ID of the Google Cloud Project.</li><li><code>DATASET</code>: BigQuery Dataset ID.</li><li><code>TABLE</code>: BigQuery Table ID within the Dataset.</li></ul></li><li><code>TBLPROPERTIES</code>:<ul><li><code>method</code>: Optional. Read method to use. Following options are available:<ul><li><code>DIRECT_READ</code>: Use the BigQuery Storage API.</li><li><code>EXPORT</code>: Export data to Google Cloud Storage in Avro format and read data files from that location.</li><li>Default is <code>DIRECT_READ</code> for Beam 2.21+ (older versions use <code>EXPORT</code>).</li></ul></li></ul></li></ul><h3 id=read-mode>Read Mode</h3><p>Beam SQL supports reading columns with simple types (<code>simpleType</code>) and arrays of simple
types (<code>ARRAY&lt;simpleType></code>).</p><p>When reading using <code>EXPORT</code> method the following pipeline options should be set:</p><ul><li><code>project</code>: ID of the Google Cloud Project.</li><li><code>tempLocation</code>: Bucket to store intermediate data in. Ex: <code>gs://temp-storage/temp</code>.</li></ul><p>When reading using <code>DIRECT_READ</code> method, an optimizer will attempt to perform
project and predicate push-down, potentially reducing the time requited to read the data from BigQuery.</p><p>More information about the BigQuery Storage API can be found <a href=/documentation/io/built-in/google-bigquery/#storage-api>here</a>.</p><h3 id=write-mode>Write Mode</h3><p>if the table does not exist, Beam creates the table specified in location when
the first record is written. If the table does exist, the specified columns must
match the existing table.</p><h3 id=schema>Schema</h3><p>Schema-related errors will cause the pipeline to crash. The Map type is not
supported. Beam SQL types map to <a href=https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types>BigQuery Standard SQL
types</a>
as follows:</p><table><tr><td>Beam SQL Type</td><td>BigQuery Standard SQL Type</td></tr><tr><td>TINYINT, SMALLINT, INTEGER, BIGINT &nbsp;</td><td>INT64</td></tr><tr><td>FLOAT, DOUBLE, DECIMAL</td><td>FLOAT64</td></tr><tr><td>BOOLEAN</td><td>BOOL</td></tr><tr><td>DATE</td><td>DATE</td></tr><tr><td>TIME</td><td>TIME</td></tr><tr><td>TIMESTAMP</td><td>TIMESTAMP</td></tr><tr><td>CHAR, VARCHAR</td><td>STRING</td></tr><tr><td>MAP</td><td>(not supported)</td></tr><tr><td>ARRAY</td><td>ARRAY</td></tr><tr><td>ROW</td><td>STRUCT</td></tr></table><h3 id=example>Example</h3><pre tabindex=0><code>CREATE EXTERNAL TABLE users (id INTEGER, username VARCHAR)
TYPE bigquery
LOCATION &#39;testing-integration:apache.users&#39;
</code></pre><h2 id=cloud-bigtable>Cloud Bigtable</h2><h3 id=syntax-2>Syntax</h3><pre tabindex=0><code>CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName (
key VARCHAR NOT NULL,
family ROW&lt;qualifier cells [, qualifier cells ]* &gt;
[, family ROW&lt; qualifier cells [, qualifier cells ]* &gt; ]*
)
TYPE bigtable
LOCATION &#39;googleapis.com/bigtable/projects/[PROJECT_ID]/instances/[INSTANCE_ID]/tables/[TABLE]&#39;
</code></pre><ul><li><code>key</code>: key of the Bigtable row</li><li><code>family</code>: name of the column family</li><li><code>qualifier</code>: the column qualifier</li><li><code>cells</code>: Either of each value:<ul><li><code>TYPE</code></li><li><code>ARRAY&lt;SIMPLE_TYPE></code></li></ul></li><li><code>LOCATION</code>:<ul><li><code>PROJECT_ID</code>: ID of the Google Cloud Project.</li><li><code>INSTANCE_ID</code>: Bigtable instance ID.</li><li><code>TABLE</code>: Bigtable Table ID.</li></ul></li><li><code>TYPE</code>: <code>SIMPLE_TYPE</code> or <code>CELL_ROW</code></li><li><code>CELL_ROW</code>: <code>ROW&lt;val SIMPLE_TYPE [, timestampMicros BIGINT [NOT NULL]] [, labels ARRAY&lt;VARCHAR> [NOT NULL]]</code></li><li><code>SIMPLE_TYPE</code>: on of the following:<ul><li><code>BINARY</code></li><li><code>VARCHAR</code></li><li><code>BIGINT</code></li><li><code>INTEGER</code></li><li><code>SMALLINT</code></li><li><code>TINYINT</code></li><li><code>DOUBLE</code></li><li><code>FLOAT</code></li><li><code>BOOLEAN</code></li><li><code>TIMESTAMP</code></li></ul></li></ul><p>An alternative syntax with a flat schema:</p><pre tabindex=0><code>CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName (
key VARCHAR NOT NULL,
qualifier SIMPLE_TYPE
[, qualifier SIMPLE_TYPE ]*
)
TYPE bigtable
LOCATION &#39;googleapis.com/bigtable/projects/[PROJECT_ID]/instances/[INSTANCE_ID]/tables/[TABLE]&#39;
TBLPROPERTIES &#39;{
&#34;columnsMapping&#34;: &#34;family:qualifier[,family:qualifier]*&#34;
}&#39;
</code></pre><ul><li><code>key</code>: key of the Bigtable row</li><li><code>family</code>: name of the column family</li><li><code>qualifier</code>: the column qualifier</li><li><code>LOCATION</code>:<ul><li><code>PROJECT_ID</code>: ID of the Google Cloud Project.</li><li><code>INSTANCE_ID</code>: Bigtable instance ID.</li><li><code>TABLE</code>: Bigtable Table ID.</li></ul></li><li><code>TBLPROPERTIES</code>: JSON object containing columnsMapping key with comma-separated
key-value pairs separated by a colon</li><li><code>SIMPLE_TYPE</code>: the same as in the previous syntax</li></ul><h3 id=read-mode-1>Read Mode</h3><p>Beam SQL supports reading rows with mandatory <code>key</code> field, at least one <code>family</code>
with at least one <code>qualifier</code>. Cells are represented as simple types (<code>SIMPLE_TYPE</code>) or
ROW type with a mandatory <code>val</code> field, optional <code>timestampMicros</code> and optional <code>labels</code>. Both
read the latest cell in the column. Cells specified as Arrays of simple types
(<code>ARRAY&lt;simpleType></code>) allow to read all the column&rsquo;s values.</p><p>For flat schema only <code>SIMPLE_TYPE</code> values are allowed. Every field except for <code>key</code> must correspond
to the key-values pairs specified in <code>columnsMapping</code>.</p><p>Not all existing column families and qualifiers have to be provided to the schema.</p><p>Filters are only allowed by <code>key</code> field with single <code>LIKE</code> statement with
<a href=https://github.com/google/re2/wiki/Syntax>RE2 Syntax</a> regex, e.g.
<code>SELECT * FROM table WHERE key LIKE '^key[012]{1}'</code></p><h3 id=write-mode-1>Write Mode</h3><p>Supported for flat schema only.</p><h3 id=example-1>Example</h3><pre tabindex=0><code>CREATE EXTERNAL TABLE beamTable(
key VARCHAR NOT NULL,
beamFamily ROW&lt;
boolLatest BOOLEAN NOT NULL,
longLatestWithTs ROW&lt;
val BIGINT NOT NULL,
timestampMicros BIGINT NOT NULL
&gt; NOT NULL,
allStrings ARRAY&lt;VARCHAR&gt; NOT NULL,
doubleLatestWithTsAndLabels ROW&lt;
val DOUBLE NOT NULL,
timestampMicros BIGINT NOT NULL,
labels ARRAY&lt;VARCHAR&gt; NOT NULL
&gt; NOT NULL,
binaryLatestWithLabels ROW&lt;
val BINARY NOT NULL,
labels ARRAY&lt;VARCHAR&gt; NOT NULL
&gt; NOT NULL
&gt; NOT NULL
)
TYPE bigtable
LOCATION &#39;googleapis.com/bigtable/projects/beam/instances/beamInstance/tables/beamTable&#39;
</code></pre><p>Flat schema example:</p><pre tabindex=0><code>CREATE EXTERNAL TABLE flatTable(
key VARCHAR NOT NULL,
boolColumn BOOLEAN NOT NULL,
longColumn BIGINT NOT NULL,
stringColumn VARCHAR NOT NULL,
doubleColumn DOUBLE NOT NULL,
binaryColumn BINARY NOT NULL
)
TYPE bigtable
LOCATION &#39;googleapis.com/bigtable/projects/beam/instances/beamInstance/tables/flatTable&#39;
TBLPROPERTIES &#39;{
&#34;columnsMapping&#34;: &#34;f:boolColumn,f:longColumn,f:stringColumn,f2:doubleColumn,f2:binaryColumn&#34;
}&#39;
</code></pre><p>Write example:</p><pre tabindex=0><code>INSERT INTO writeTable(key, boolColumn, longColumn, stringColumn, doubleColumn)
VALUES (&#39;key&#39;, TRUE, 10, &#39;stringValue&#39;, 5.5)
</code></pre><h2 id=pubsub>Pub/Sub</h2><h3 id=syntax-3>Syntax</h3><h4 id=nested-mode>Nested mode</h4><pre tabindex=0><code>CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName(
event_timestamp TIMESTAMP,
attributes [MAP&lt;VARCHAR, VARCHAR&gt;, ARRAY&lt;ROW&lt;VARCHAR key, VARCHAR value&gt;&gt;],
payload [BYTES, ROW&lt;tableElement [, tableElement ]*&gt;]
)
TYPE pubsub
LOCATION &#39;projects/[PROJECT]/topics/[TOPIC]&#39;
</code></pre><h4 id=flattened-mode>Flattened mode</h4><pre tabindex=0><code>CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName(tableElement [, tableElement ]*)
TYPE pubsub
LOCATION &#39;projects/[PROJECT]/topics/[TOPIC]&#39;
</code></pre><p>In nested mode, the following fields hold topic metadata. The presence of the
<code>attributes</code> field triggers nested mode usage.</p><ul><li><p><code>event_timestamp</code>: The event timestamp associated with the Pub/Sub message
by PubsubIO. It can be one of the following:</p><ul><li>Message publish time, which is provided by Pub/Sub. This is the default
value if no extra configuration is provided.</li><li>A timestamp specified in one of the user-provided message attributes.
The attribute key is configured by the <code>timestampAttributeKey</code> field of
the <code>tblProperties</code> blob. The value of the attribute should conform to
the <a href=https://beam.apache.org/releases/javadoc/2.4.0/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.Read.html#withTimestampAttribute-java.lang.String->requirements of
PubsubIO</a>,
which is either millis since Unix epoch or <a href=https://www.ietf.org/rfc/rfc3339.txt>RFC 339
</a>date string.</li></ul></li><li><p><code>attributes</code>: The user-provided attributes map from the Pub/Sub message;</p></li><li><p><code>payload</code>: The schema of the payload of the Pub/Sub message. If a record
can&rsquo;t be unmarshalled, the record is written to the topic specified in the
<code>deadLeaderQueue</code> field of the <code>tblProperties</code> blob. If no dead-letter queue
is specified in this case, an exception is thrown and the pipeline will
crash.</p></li><li><p><code>LOCATION</code>:</p><ul><li><code>PROJECT</code>: ID of the Google Cloud Project</li><li><code>TOPIC</code>: The Pub/Sub topic name. A subscription will be created
automatically, but the subscription is not cleaned up automatically.
Specifying an existing subscription is not supported.</li></ul></li><li><p><code>TBLPROPERTIES</code>:</p><ul><li><code>timestampAttributeKey</code>: Optional. The key which contains the event
timestamp associated with the Pub/Sub message. If not specified, the
message publish timestamp is used as an event timestamp for
windowing/watermarking.</li><li><code>deadLetterQueue</code>: The topic into which messages are written if the
payload was not parsed. If not specified, an exception is thrown for
parsing failures.</li><li><code>format</code>: Optional. Allows you to specify the Pubsub payload format.</li></ul></li></ul><h3 id=read-mode-2>Read Mode</h3><p>PubsubIO supports reading from topics by creating a new subscription.</p><h3 id=write-mode-2>Write Mode</h3><p>PubsubIO supports writing to topics.</p><h3 id=schema-1>Schema</h3><p>Pub/Sub messages have metadata associated with them, and you can reference this
metadata in your queries. For each message, Pub/Sub exposes its publish time and
a map of user-provided attributes in addition to the payload (unstructured in
the general case). This information must be preserved and accessible from the
SQL statements. Currently, this means that PubsubIO tables require you to
declare a special set of columns, as shown below.</p><h3 id=supported-payload>Supported Payload</h3><ul><li>Pub/Sub supports <a href=#generic-payload-handling>Generic Payload Handling</a>.</li></ul><h3 id=example-2>Example</h3><pre tabindex=0><code>CREATE EXTERNAL TABLE locations (event_timestamp TIMESTAMP, attributes MAP&lt;VARCHAR, VARCHAR&gt;, payload ROW&lt;id INTEGER, location VARCHAR&gt;)
TYPE pubsub
LOCATION &#39;projects/testing-integration/topics/user-location&#39;
</code></pre><h2 id=pubsub-lite>Pub/Sub Lite</h2><h3 id=syntax-4>Syntax</h3><pre tabindex=0><code>CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName(
publish_timestamp DATETIME,
event_timestamp DATETIME,
message_key BYTES,
attributes ARRAY&lt;ROW&lt;key VARCHAR, `values` ARRAY&lt;VARBINARY&gt;&gt;&gt;,
payload [BYTES, ROW&lt;tableElement [, tableElement ]*&gt;]
)
TYPE pubsublite
// For writing
LOCATION &#39;projects/[PROJECT]/locations/[GCP-LOCATION]/topics/[TOPIC]&#39;
// For reading
LOCATION &#39;projects/[PROJECT]/locations/[GCP-LOCATION]/subscriptions/[SUBSCRIPTION]&#39;
</code></pre><ul><li><code>LOCATION</code>:<ul><li><code>PROJECT</code>: ID of the Google Cloud Project</li><li><code>TOPIC</code>: The Pub/Sub Lite topic name.</li><li><code>SUBSCRIPTION</code>: The Pub/Sub Lite subscription name.</li><li><code>GCP-LOCATION</code>: The location for this Pub/Sub Lite topic os subscription.</li></ul></li><li><code>TBLPROPERTIES</code>:<ul><li><code>timestampAttributeKey</code>: Optional. The key which contains the event
timestamp associated with the Pub/Sub message. If not specified, the
message publish timestamp is used as an event timestamp for
windowing/watermarking.</li><li><code>deadLetterQueue</code>: Optional, supports
<a href=#generic-dlq-handling>Generic DLQ Handling</a></li><li><code>format</code>: Optional. Allows you to specify the payload format.</li></ul></li></ul><h3 id=read-mode-3>Read Mode</h3><p>PubsubLiteIO supports reading from subscriptions.</p><h3 id=write-mode-3>Write Mode</h3><p>PubsubLiteIO supports writing to topics.</p><h3 id=supported-payload-1>Supported Payload</h3><ul><li>Pub/Sub Lite supports <a href=#generic-payload-handling>Generic Payload Handling</a>.</li></ul><h3 id=example-3>Example</h3><pre tabindex=0><code>CREATE EXTERNAL TABLE locations (event_timestamp TIMESTAMP, attributes ARRAY&lt;ROW&lt;key VARCHAR, `values` ARRAY&lt;VARBINARY&gt;&gt;&gt;, payload ROW&lt;id INTEGER, location VARCHAR&gt;)
TYPE pubsublite
LOCATION &#39;projects/testing-integration/locations/us-central1-a/topics/user-location&#39;
</code></pre><h2 id=kafka>Kafka</h2><p>KafkaIO is experimental in Beam SQL.</p><h3 id=syntax-5>Syntax</h3><h4 id=flattened-mode-1>Flattened mode</h4><pre tabindex=0><code>CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName (tableElement [, tableElement ]*)
TYPE kafka
LOCATION &#39;my.company.url.com:2181/topic1&#39;
TBLPROPERTIES &#39;{
&#34;bootstrap_servers&#34;: [&#34;localhost:9092&#34;, &#34;PLAINTEXT://192.168.1.200:2181&#34;],
&#34;topics&#34;: [&#34;topic2&#34;, &#34;topic3&#34;],
&#34;format&#34;: &#34;json&#34;
}&#39;
</code></pre><h4 id=nested-mode-1>Nested mode</h4><pre tabindex=0><code>CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName (
event_timestamp DATETIME,
message_key BYTES,
headers ARRAY&lt;ROW&lt;key VARCHAR, `values` ARRAY&lt;VARBINARY&gt;&gt;&gt;,
payload [BYTES, ROW&lt;tableElement [, tableElement ]*&gt;]
)
TYPE kafka
LOCATION &#39;my.company.url.com:2181/topic1&#39;
TBLPROPERTIES &#39;{
&#34;bootstrap_servers&#34;: [&#34;localhost:9092&#34;, &#34;PLAINTEXT://192.168.1.200:2181&#34;],
&#34;topics&#34;: [&#34;topic2&#34;, &#34;topic3&#34;],
&#34;format&#34;: &#34;json&#34;
}&#39;
</code></pre><p>The presence of the <code>headers</code> field triggers nested mode usage.</p><ul><li><code>LOCATION</code>: A url with the initial bootstrap broker to use and the initial
topic name provided as the path.</li><li><code>TBLPROPERTIES</code>:<ul><li><code>bootstrap_servers</code>: Optional. Allows you to specify additional
bootstrap servers, which are used in addition to the one in <code>LOCATION</code>.</li><li><code>topics</code>: Optional. Allows you to specify additional topics, which are
used in addition to the one in <code>LOCATION</code>.</li><li><code>format</code>: Optional. Allows you to specify the Kafka values format. Possible values are
{<code>csv</code>, <code>avro</code>, <code>json</code>, <code>proto</code>, <code>thrift</code>}. Defaults to <code>csv</code> in
flattened mode or <code>json</code> in nested mode. <code>csv</code> does not support nested
mode.</li></ul></li></ul><h3 id=read-mode-4>Read Mode</h3><p>Read Mode supports reading from a topic.</p><h3 id=write-mode-4>Write Mode</h3><p>Write Mode supports writing to a topic.</p><h3 id=supported-formats>Supported Formats</h3><ul><li>CSV (default)<ul><li>Beam parses the messages, attempting to parse fields according to the
types specified in the schema.</li></ul></li><li>Kafka supports all <a href=#generic-payload-handling>Generic Payload Handling</a>
formats.</li></ul><h3 id=schema-2>Schema</h3><p>For CSV only simple types are supported.</p><h2 id=mongodb>MongoDB</h2><h3 id=syntax-6>Syntax</h3><pre tabindex=0><code>CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName (tableElement [, tableElement ]*)
TYPE mongodb
LOCATION &#39;mongodb://[HOST]:[PORT]/[DATABASE]/[COLLECTION]&#39;
</code></pre><ul><li><code>LOCATION</code>: Location of the collection.<ul><li><code>HOST</code>: Location of the MongoDB server. Can be localhost or an ip address.
When authentication is required username and password can be specified
as follows: <code>username:password@localhost</code>.</li><li><code>PORT</code>: Port on which MongoDB server is listening.</li><li><code>DATABASE</code>: Database to connect to.</li><li><code>COLLECTION</code>: Collection within the database.</li></ul></li></ul><h3 id=read-mode-5>Read Mode</h3><p>Read Mode supports reading from a collection.</p><h3 id=write-mode-5>Write Mode</h3><p>Write Mode supports writing to a collection.</p><h3 id=schema-3>Schema</h3><p>Only simple types are supported. MongoDB documents are mapped to Beam SQL types via <a href=https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/transforms/JsonToRow.html><code>JsonToRow</code></a> transform.</p><h3 id=example-4>Example</h3><pre tabindex=0><code>CREATE EXTERNAL TABLE users (id INTEGER, username VARCHAR)
TYPE mongodb
LOCATION &#39;mongodb://localhost:27017/apache/users&#39;
</code></pre><h2 id=text>Text</h2><p>TextIO is experimental in Beam SQL. Read Mode and Write Mode do not currently
access the same underlying data.</p><h3 id=syntax-7>Syntax</h3><pre tabindex=0><code>CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName (tableElement [, tableElement ]*)
TYPE text
LOCATION &#39;/home/admin/orders&#39;
TBLPROPERTIES &#39;{&#34;format: &#34;Excel&#34;}&#39;
</code></pre><ul><li><code>LOCATION</code>: The path to the file for Read Mode. The prefix for Write Mode.</li><li><code>TBLPROPERTIES</code>:<ul><li><code>format</code>: Optional. Allows you to specify the CSV Format, which controls
the field delimeter, quote character, record separator, and other properties.
See the following table:</li></ul></li></ul><div class=table-container-wrapper><div class=table-bordered-wrapper><table><thead><tr><th>Value for <code>format</code></th><th>Field delimiter</th><th>Quote</th><th>Record separator</th><th>Ignore empty lines?</th><th>Allow missing column names?</th></tr></thead><tbody><tr><td><code>default</code></td><td><code>,</code></td><td><code>"</code></td><td><code>\r\n</code></td><td>Yes</td><td>No</td></tr><tr><td><code>rfc4180</code></td><td><code>,</code></td><td><code>"</code></td><td><code>\r\n</code></td><td>No</td><td>No</td></tr><tr><td><code>excel</code></td><td><code>,</code></td><td><code>"</code></td><td><code>\r\n</code></td><td>No</td><td>Yes</td></tr><tr><td><code>tdf</code></td><td><code>\t</code></td><td><code>"</code></td><td><code>\r\n</code></td><td>Yes</td><td>No</td></tr><tr><td><code>mysql</code></td><td><code>\t</code></td><td>none</td><td><code>\n</code></td><td>No</td><td>No</td></tr></tbody></table></div></div><h3 id=read-mode-6>Read Mode</h3><p>Read Mode supports reading from a file.</p><h3 id=write-mode-6>Write Mode</h3><p>Write Mode supports writing to a set of files. TextIO creates file on writes.</p><h3 id=supported-payload-2>Supported Payload</h3><ul><li>CSV<ul><li>Beam parses the messages, attempting to parse fields according to the
types specified in the schema using org.apache.commons.csv.</li></ul></li></ul><h3 id=schema-4>Schema</h3><p>Only simple types are supported.</p><h3 id=example-5>Example</h3><pre tabindex=0><code>CREATE EXTERNAL TABLE orders (id INTEGER, price INTEGER)
TYPE text
LOCATION &#39;/home/admin/orders&#39;
</code></pre><h2 id=generic-payload-handling>Generic Payload Handling</h2><p>Certain data sources and sinks support generic payload handling. This handling
parses a byte array payload field into a table schema. The following schemas are
supported by this handling. All require at least setting <code>"format": "&lt;type>"</code>,
and may require other properties.</p><ul><li><code>avro</code>: Avro<ul><li>An Avro schema is automatically generated from the specified field
types. It is used to parse incoming messages and to format outgoing
messages.</li></ul></li><li><code>json</code>: JSON Objects<ul><li>Beam attempts to parse the byte array as UTF-8 JSON to match the schema.</li></ul></li><li><code>proto</code>: Protocol Buffers<ul><li>Beam locates the equivalent Protocol Buffer class and uses it to parse
the payload</li><li><code>protoClass</code>: Required. The proto class name to use. Must be built into
the deployed JAR.</li><li>Fields in the schema have to match the fields of the given <code>protoClass</code>.</li></ul></li><li><code>thrift</code>: Thrift<ul><li>Fields in the schema have to match the fields of the given
<code>thriftClass</code>.</li><li><code>thriftClass</code>: Required. Allows you to specify full thrift java class
name. Must be built into the deployed JAR.</li><li><code>thriftProtocolFactoryClass</code>: Required. Allows you to specify full class
name of the <code>TProtocolFactory</code> to use for thrift serialization. Must be
built into the deployed JAR.</li><li>The <code>TProtocolFactory</code> used for thrift serialization must match the
provided <code>thriftProtocolFactoryClass</code>.</li></ul></li></ul><h2 id=generic-dlq-handling>Generic DLQ Handling</h2><p>Sources and sinks which support generic DLQ handling specify a parameter with
the format <code>"&lt;dlqParamName>": "[DLQ_KIND]:[DLQ_ID]"</code>. The following types of
DLQ handling are supported:</p><ul><li><code>bigquery</code>: BigQuery<ul><li>DLQ_ID is the table spec for an output table with an &ldquo;error&rdquo; string
field and &ldquo;payload&rdquo; byte array field.</li></ul></li><li><code>pubsub</code>: Pub/Sub Topic<ul><li>DLQ_ID is the full path of the Pub/Sub Topic.</li></ul></li><li><code>pubsublite</code>: Pub/Sub Lite Topic<ul><li>DLQ_ID is the full path of the Pub/Sub Lite Topic.</li></ul></li></ul></div></div><footer class=footer><div class=footer__contained><div class=footer__cols><div class="footer__cols__col footer__cols__col__logos"><div class=footer__cols__col__logo><img src=/images/beam_logo_circle.svg class=footer__logo alt="Beam logo"></div><div class=footer__cols__col__logo><img src=/images/apache_logo_circle.svg class=footer__logo alt="Apache logo"></div></div><div class=footer-wrapper><div class=wrapper-grid><div class=footer__cols__col><div class=footer__cols__col__title>Start</div><div class=footer__cols__col__link><a href=/get-started/beam-overview/>Overview</a></div><div class=footer__cols__col__link><a href=/get-started/quickstart-java/>Quickstart (Java)</a></div><div class=footer__cols__col__link><a href=/get-started/quickstart-py/>Quickstart (Python)</a></div><div class=footer__cols__col__link><a href=/get-started/quickstart-go/>Quickstart (Go)</a></div><div class=footer__cols__col__link><a href=/get-started/downloads/>Downloads</a></div></div><div class=footer__cols__col><div class=footer__cols__col__title>Docs</div><div class=footer__cols__col__link><a href=/documentation/programming-guide/>Concepts</a></div><div class=footer__cols__col__link><a href=/documentation/pipelines/design-your-pipeline/>Pipelines</a></div><div class=footer__cols__col__link><a href=/documentation/runners/capability-matrix/>Runners</a></div></div><div class=footer__cols__col><div class=footer__cols__col__title>Community</div><div class=footer__cols__col__link><a href=/contribute/>Contribute</a></div><div class=footer__cols__col__link><a href=https://projects.apache.org/committee.html?beam target=_blank>Team<img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></div><div class=footer__cols__col__link><a href=/community/presentation-materials/>Media</a></div><div class=footer__cols__col__link><a href=/community/in-person/>Events/Meetups</a></div><div class=footer__cols__col__link><a href=/community/contact-us/>Contact Us</a></div></div><div class=footer__cols__col><div class=footer__cols__col__title>Resources</div><div class=footer__cols__col__link><a href=/blog/>Blog</a></div><div class=footer__cols__col__link><a href=https://github.com/apache/beam>GitHub</a></div></div></div><div class=footer__bottom>&copy;
<a href=https://www.apache.org>The Apache Software Foundation</a>
| <a href=/privacy_policy>Privacy Policy</a>
| <a href=/feed.xml>RSS Feed</a><br><br>Apache Beam, Apache, Beam, the Beam logo, and the Apache feather logo are either registered trademarks or trademarks of The Apache Software Foundation. All other products or name brands are trademarks of their respective holders, including The Apache Software Foundation.</div></div><div class="footer__cols__col footer__cols__col__logos"><div class=footer__cols__col--group><div class=footer__cols__col__logo><a href=https://github.com/apache/beam><img src=/images/logos/social-icons/github-logo-150.png class=footer__logo alt="Github logo"></a></div><div class=footer__cols__col__logo><a href=https://www.linkedin.com/company/apache-beam/><img src=/images/logos/social-icons/linkedin-logo-150.png class=footer__logo alt="Linkedin logo"></a></div></div><div class=footer__cols__col--group><div class=footer__cols__col__logo><a href=https://twitter.com/apachebeam><img src=/images/logos/social-icons/twitter-logo-150.png class=footer__logo alt="Twitter logo"></a></div><div class=footer__cols__col__logo><a href=https://www.youtube.com/channel/UChNnb_YO_7B0HlW6FhAXZZQ><img src=/images/logos/social-icons/youtube-logo-150.png class=footer__logo alt="Youtube logo"></a></div></div></div></div></div></footer></body></html>