blob: 917b4f85283adc1b663d3a53744c39fe764d14ba [file] [log] [blame]
<!DOCTYPE html>
<html lang="en" data-content_root="./">
<head>
<meta charset="utf-8" />
<meta name="viewport" content="width=device-width, initial-scale=1.0" /><meta name="viewport" content="width=device-width, initial-scale=1" />
<title>Arrow Flight &#8212; Apache Arrow C++ Cookbook documentation</title>
<link rel="stylesheet" type="text/css" href="_static/pygments.css?v=d1102ebc" />
<link rel="stylesheet" type="text/css" href="_static/alabaster.css?v=49eeb2a1" />
<script src="_static/documentation_options.js?v=5929fcd5"></script>
<script src="_static/doctools.js?v=888ff710"></script>
<script src="_static/sphinx_highlight.js?v=dc90522c"></script>
<link rel="icon" href="_static/favicon.ico"/>
<link rel="index" title="Index" href="genindex.html" />
<link rel="search" title="Search" href="search.html" />
<link rel="prev" title="Reading and Writing Datasets" href="datasets.html" />
<link rel="stylesheet" href="_static/custom.css" type="text/css" />
<!-- Matomo -->
<script>
var _paq = window._paq = window._paq || [];
/* tracker methods like "setCustomDimension" should be called before "trackPageView" */
/* We explicitly disable cookie tracking to avoid privacy issues */
_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', '20']);
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>
<div class="document">
<div class="documentwrapper">
<div class="bodywrapper">
<div class="body" role="main">
<section id="arrow-flight">
<h1><a class="toc-backref" href="#id15" role="doc-backlink">Arrow Flight</a><a class="headerlink" href="#arrow-flight" title="Link to this heading"></a></h1>
<p>This section contains a number of recipes for working with Arrow
Flight, an RPC library specialized for tabular datasets. For more
about Flight, see <span class="xref std std-doc">format/Flight</span>.</p>
<nav class="contents" id="contents">
<p class="topic-title">Contents</p>
<ul class="simple">
<li><p><a class="reference internal" href="#arrow-flight" id="id15">Arrow Flight</a></p>
<ul>
<li><p><a class="reference internal" href="#simple-parquet-storage-service-with-arrow-flight" id="id16">Simple Parquet storage service with Arrow Flight</a></p></li>
<li><p><a class="reference internal" href="#setting-grpc-client-options" id="id17">Setting gRPC client options</a></p></li>
<li><p><a class="reference internal" href="#flight-service-with-other-grpc-endpoints" id="id18">Flight Service with other gRPC endpoints</a></p>
<ul>
<li><p><a class="reference internal" href="#creating-the-server" id="id19">Creating the server</a></p></li>
<li><p><a class="reference internal" href="#creating-the-client" id="id20">Creating the client</a></p></li>
</ul>
</li>
</ul>
</li>
</ul>
</nav>
<section id="simple-parquet-storage-service-with-arrow-flight">
<h2><a class="toc-backref" href="#id16" role="doc-backlink">Simple Parquet storage service with Arrow Flight</a><a class="headerlink" href="#simple-parquet-storage-service-with-arrow-flight" title="Link to this heading"></a></h2>
<p>We’ll implement a service that provides a key-value store for tabular
data, using Flight to handle uploads/requests and Parquet to store the
actual data.</p>
<p>First, we’ll implement the service itself. For simplicity, we won’t
use the <a class="reference internal" href="datasets.html"><span class="doc">Datasets</span></a> API in favor of just using the
Parquet API directly.</p>
<div class="literal-block-wrapper docutils container" id="id1">
<div class="code-block-caption"><span class="caption-text">Parquet storage service, server implementation</span><a class="headerlink" href="#id1" title="Link to this code"></a></div>
<div class="highlight-cpp notranslate"><div class="highlight"><pre><span></span><span class="linenos"> 1</span><span class="k">class</span><span class="w"> </span><span class="nc">ParquetStorageService</span><span class="w"> </span><span class="o">:</span><span class="w"> </span><span class="k">public</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightServerBase</span><span class="w"> </span><span class="p">{</span>
<span class="linenos"> 2</span><span class="w"> </span><span class="k">public</span><span class="o">:</span>
<span class="linenos"> 3</span><span class="w"> </span><span class="k">const</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">ActionType</span><span class="w"> </span><span class="n">kActionDropDataset</span><span class="p">{</span><span class="s">&quot;drop_dataset&quot;</span><span class="p">,</span><span class="w"> </span><span class="s">&quot;Delete a dataset.&quot;</span><span class="p">};</span>
<span class="linenos"> 4</span>
<span class="linenos"> 5</span><span class="w"> </span><span class="k">explicit</span><span class="w"> </span><span class="n">ParquetStorageService</span><span class="p">(</span><span class="n">std</span><span class="o">::</span><span class="n">shared_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">fs</span><span class="o">::</span><span class="n">FileSystem</span><span class="o">&gt;</span><span class="w"> </span><span class="n">root</span><span class="p">)</span>
<span class="linenos"> 6</span><span class="w"> </span><span class="o">:</span><span class="w"> </span><span class="n">root_</span><span class="p">(</span><span class="n">std</span><span class="o">::</span><span class="n">move</span><span class="p">(</span><span class="n">root</span><span class="p">))</span><span class="w"> </span><span class="p">{}</span>
<span class="linenos"> 7</span>
<span class="linenos"> 8</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Status</span><span class="w"> </span><span class="n">ListFlights</span><span class="p">(</span>
<span class="linenos"> 9</span><span class="w"> </span><span class="k">const</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">ServerCallContext</span><span class="o">&amp;</span><span class="p">,</span><span class="w"> </span><span class="k">const</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">Criteria</span><span class="o">*</span><span class="p">,</span>
<span class="linenos"> 10</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightListing</span><span class="o">&gt;*</span><span class="w"> </span><span class="n">listings</span><span class="p">)</span><span class="w"> </span><span class="k">override</span><span class="w"> </span><span class="p">{</span>
<span class="linenos"> 11</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">fs</span><span class="o">::</span><span class="n">FileSelector</span><span class="w"> </span><span class="n">selector</span><span class="p">;</span>
<span class="linenos"> 12</span><span class="w"> </span><span class="n">selector</span><span class="p">.</span><span class="n">base_dir</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s">&quot;/&quot;</span><span class="p">;</span>
<span class="linenos"> 13</span><span class="w"> </span><span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="k">auto</span><span class="w"> </span><span class="n">listing</span><span class="p">,</span><span class="w"> </span><span class="n">root_</span><span class="o">-&gt;</span><span class="n">GetFileInfo</span><span class="p">(</span><span class="n">selector</span><span class="p">));</span>
<span class="linenos"> 14</span>
<span class="linenos"> 15</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">vector</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightInfo</span><span class="o">&gt;</span><span class="w"> </span><span class="n">flights</span><span class="p">;</span>
<span class="linenos"> 16</span><span class="w"> </span><span class="k">for</span><span class="w"> </span><span class="p">(</span><span class="k">const</span><span class="w"> </span><span class="k">auto</span><span class="o">&amp;</span><span class="w"> </span><span class="n">file_info</span><span class="w"> </span><span class="o">:</span><span class="w"> </span><span class="n">listing</span><span class="p">)</span><span class="w"> </span><span class="p">{</span>
<span class="linenos"> 17</span><span class="w"> </span><span class="k">if</span><span class="w"> </span><span class="p">(</span><span class="o">!</span><span class="n">file_info</span><span class="p">.</span><span class="n">IsFile</span><span class="p">()</span><span class="w"> </span><span class="o">||</span><span class="w"> </span><span class="n">file_info</span><span class="p">.</span><span class="n">extension</span><span class="p">()</span><span class="w"> </span><span class="o">!=</span><span class="w"> </span><span class="s">&quot;parquet&quot;</span><span class="p">)</span><span class="w"> </span><span class="k">continue</span><span class="p">;</span>
<span class="linenos"> 18</span><span class="w"> </span><span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="k">auto</span><span class="w"> </span><span class="n">info</span><span class="p">,</span><span class="w"> </span><span class="n">MakeFlightInfo</span><span class="p">(</span><span class="n">file_info</span><span class="p">));</span>
<span class="linenos"> 19</span><span class="w"> </span><span class="n">flights</span><span class="p">.</span><span class="n">push_back</span><span class="p">(</span><span class="n">std</span><span class="o">::</span><span class="n">move</span><span class="p">(</span><span class="n">info</span><span class="p">));</span>
<span class="linenos"> 20</span><span class="w"> </span><span class="p">}</span>
<span class="linenos"> 21</span>
<span class="linenos"> 22</span><span class="w"> </span><span class="o">*</span><span class="n">listings</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightListing</span><span class="o">&gt;</span><span class="p">(</span>
<span class="linenos"> 23</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">SimpleFlightListing</span><span class="p">(</span><span class="n">std</span><span class="o">::</span><span class="n">move</span><span class="p">(</span><span class="n">flights</span><span class="p">)));</span>
<span class="linenos"> 24</span><span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Status</span><span class="o">::</span><span class="n">OK</span><span class="p">();</span>
<span class="linenos"> 25</span><span class="w"> </span><span class="p">}</span>
<span class="linenos"> 26</span>
<span class="linenos"> 27</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Status</span><span class="w"> </span><span class="n">GetFlightInfo</span><span class="p">(</span><span class="k">const</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">ServerCallContext</span><span class="o">&amp;</span><span class="p">,</span>
<span class="linenos"> 28</span><span class="w"> </span><span class="k">const</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightDescriptor</span><span class="o">&amp;</span><span class="w"> </span><span class="n">descriptor</span><span class="p">,</span>
<span class="linenos"> 29</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightInfo</span><span class="o">&gt;*</span><span class="w"> </span><span class="n">info</span><span class="p">)</span><span class="w"> </span><span class="k">override</span><span class="w"> </span><span class="p">{</span>
<span class="linenos"> 30</span><span class="w"> </span><span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="k">auto</span><span class="w"> </span><span class="n">file_info</span><span class="p">,</span><span class="w"> </span><span class="n">FileInfoFromDescriptor</span><span class="p">(</span><span class="n">descriptor</span><span class="p">));</span>
<span class="linenos"> 31</span><span class="w"> </span><span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="k">auto</span><span class="w"> </span><span class="n">flight_info</span><span class="p">,</span><span class="w"> </span><span class="n">MakeFlightInfo</span><span class="p">(</span><span class="n">file_info</span><span class="p">));</span>
<span class="linenos"> 32</span><span class="w"> </span><span class="o">*</span><span class="n">info</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightInfo</span><span class="o">&gt;</span><span class="p">(</span>
<span class="linenos"> 33</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightInfo</span><span class="p">(</span><span class="n">std</span><span class="o">::</span><span class="n">move</span><span class="p">(</span><span class="n">flight_info</span><span class="p">)));</span>
<span class="linenos"> 34</span><span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Status</span><span class="o">::</span><span class="n">OK</span><span class="p">();</span>
<span class="linenos"> 35</span><span class="w"> </span><span class="p">}</span>
<span class="linenos"> 36</span>
<span class="linenos"> 37</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Status</span><span class="w"> </span><span class="n">DoPut</span><span class="p">(</span><span class="k">const</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">ServerCallContext</span><span class="o">&amp;</span><span class="p">,</span>
<span class="linenos"> 38</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightMessageReader</span><span class="o">&gt;</span><span class="w"> </span><span class="n">reader</span><span class="p">,</span>
<span class="linenos"> 39</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightMetadataWriter</span><span class="o">&gt;</span><span class="p">)</span><span class="w"> </span><span class="k">override</span><span class="w"> </span><span class="p">{</span>
<span class="linenos"> 40</span><span class="w"> </span><span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="k">auto</span><span class="w"> </span><span class="n">file_info</span><span class="p">,</span><span class="w"> </span><span class="n">FileInfoFromDescriptor</span><span class="p">(</span><span class="n">reader</span><span class="o">-&gt;</span><span class="n">descriptor</span><span class="p">()));</span>
<span class="linenos"> 41</span><span class="w"> </span><span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="k">auto</span><span class="w"> </span><span class="n">sink</span><span class="p">,</span><span class="w"> </span><span class="n">root_</span><span class="o">-&gt;</span><span class="n">OpenOutputStream</span><span class="p">(</span><span class="n">file_info</span><span class="p">.</span><span class="n">path</span><span class="p">()));</span>
<span class="linenos"> 42</span><span class="w"> </span><span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">std</span><span class="o">::</span><span class="n">shared_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">Table</span><span class="o">&gt;</span><span class="w"> </span><span class="n">table</span><span class="p">,</span><span class="w"> </span><span class="n">reader</span><span class="o">-&gt;</span><span class="n">ToTable</span><span class="p">());</span>
<span class="linenos"> 43</span>
<span class="linenos"> 44</span><span class="w"> </span><span class="n">ARROW_RETURN_NOT_OK</span><span class="p">(</span><span class="n">parquet</span><span class="o">::</span><span class="n">arrow</span><span class="o">::</span><span class="n">WriteTable</span><span class="p">(</span><span class="o">*</span><span class="n">table</span><span class="p">,</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">default_memory_pool</span><span class="p">(),</span>
<span class="linenos"> 45</span><span class="w"> </span><span class="n">sink</span><span class="p">,</span><span class="w"> </span><span class="cm">/*chunk_size=*/</span><span class="mi">65536</span><span class="p">));</span>
<span class="linenos"> 46</span><span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Status</span><span class="o">::</span><span class="n">OK</span><span class="p">();</span>
<span class="linenos"> 47</span><span class="w"> </span><span class="p">}</span>
<span class="linenos"> 48</span>
<span class="linenos"> 49</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Status</span><span class="w"> </span><span class="n">DoGet</span><span class="p">(</span><span class="k">const</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">ServerCallContext</span><span class="o">&amp;</span><span class="p">,</span>
<span class="linenos"> 50</span><span class="w"> </span><span class="k">const</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">Ticket</span><span class="o">&amp;</span><span class="w"> </span><span class="n">request</span><span class="p">,</span>
<span class="linenos"> 51</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightDataStream</span><span class="o">&gt;*</span><span class="w"> </span><span class="n">stream</span><span class="p">)</span><span class="w"> </span><span class="k">override</span><span class="w"> </span><span class="p">{</span>
<span class="linenos"> 52</span><span class="w"> </span><span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="k">auto</span><span class="w"> </span><span class="n">input</span><span class="p">,</span><span class="w"> </span><span class="n">root_</span><span class="o">-&gt;</span><span class="n">OpenInputFile</span><span class="p">(</span><span class="n">request</span><span class="p">.</span><span class="n">ticket</span><span class="p">));</span>
<span class="linenos"> 53</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">parquet</span><span class="o">::</span><span class="n">arrow</span><span class="o">::</span><span class="n">FileReader</span><span class="o">&gt;</span><span class="w"> </span><span class="n">reader</span><span class="p">;</span>
<span class="linenos"> 54</span><span class="w"> </span><span class="n">ARROW_RETURN_NOT_OK</span><span class="p">(</span><span class="n">parquet</span><span class="o">::</span><span class="n">arrow</span><span class="o">::</span><span class="n">OpenFile</span><span class="p">(</span><span class="n">std</span><span class="o">::</span><span class="n">move</span><span class="p">(</span><span class="n">input</span><span class="p">),</span>
<span class="linenos"> 55</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">default_memory_pool</span><span class="p">(),</span><span class="w"> </span><span class="o">&amp;</span><span class="n">reader</span><span class="p">));</span>
<span class="linenos"> 56</span>
<span class="linenos"> 57</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">shared_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">Table</span><span class="o">&gt;</span><span class="w"> </span><span class="n">table</span><span class="p">;</span>
<span class="linenos"> 58</span><span class="w"> </span><span class="n">ARROW_RETURN_NOT_OK</span><span class="p">(</span><span class="n">reader</span><span class="o">-&gt;</span><span class="n">ReadTable</span><span class="p">(</span><span class="o">&amp;</span><span class="n">table</span><span class="p">));</span>
<span class="linenos"> 59</span><span class="w"> </span><span class="c1">// Note that we can&#39;t directly pass TableBatchReader to</span>
<span class="linenos"> 60</span><span class="w"> </span><span class="c1">// RecordBatchStream because TableBatchReader keeps a non-owning</span>
<span class="linenos"> 61</span><span class="w"> </span><span class="c1">// reference to the underlying Table, which would then get freed</span>
<span class="linenos"> 62</span><span class="w"> </span><span class="c1">// when we exit this function</span>
<span class="linenos"> 63</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">vector</span><span class="o">&lt;</span><span class="n">std</span><span class="o">::</span><span class="n">shared_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">RecordBatch</span><span class="o">&gt;&gt;</span><span class="w"> </span><span class="n">batches</span><span class="p">;</span>
<span class="linenos"> 64</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">TableBatchReader</span><span class="w"> </span><span class="nf">batch_reader</span><span class="p">(</span><span class="o">*</span><span class="n">table</span><span class="p">);</span>
<span class="linenos"> 65</span><span class="w"> </span><span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">batches</span><span class="p">,</span><span class="w"> </span><span class="n">batch_reader</span><span class="p">.</span><span class="n">ToRecordBatches</span><span class="p">());</span>
<span class="linenos"> 66</span>
<span class="linenos"> 67</span><span class="w"> </span><span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="k">auto</span><span class="w"> </span><span class="n">owning_reader</span><span class="p">,</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">RecordBatchReader</span><span class="o">::</span><span class="n">Make</span><span class="p">(</span>
<span class="linenos"> 68</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">move</span><span class="p">(</span><span class="n">batches</span><span class="p">),</span><span class="w"> </span><span class="n">table</span><span class="o">-&gt;</span><span class="n">schema</span><span class="p">()));</span>
<span class="linenos"> 69</span><span class="w"> </span><span class="o">*</span><span class="n">stream</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightDataStream</span><span class="o">&gt;</span><span class="p">(</span>
<span class="linenos"> 70</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">RecordBatchStream</span><span class="p">(</span><span class="n">owning_reader</span><span class="p">));</span>
<span class="linenos"> 71</span>
<span class="linenos"> 72</span><span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Status</span><span class="o">::</span><span class="n">OK</span><span class="p">();</span>
<span class="linenos"> 73</span><span class="w"> </span><span class="p">}</span>
<span class="linenos"> 74</span>
<span class="linenos"> 75</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Status</span><span class="w"> </span><span class="n">ListActions</span><span class="p">(</span><span class="k">const</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">ServerCallContext</span><span class="o">&amp;</span><span class="p">,</span>
<span class="linenos"> 76</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">vector</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">ActionType</span><span class="o">&gt;*</span><span class="w"> </span><span class="n">actions</span><span class="p">)</span><span class="w"> </span><span class="k">override</span><span class="w"> </span><span class="p">{</span>
<span class="linenos"> 77</span><span class="w"> </span><span class="o">*</span><span class="n">actions</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="p">{</span><span class="n">kActionDropDataset</span><span class="p">};</span>
<span class="linenos"> 78</span><span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Status</span><span class="o">::</span><span class="n">OK</span><span class="p">();</span>
<span class="linenos"> 79</span><span class="w"> </span><span class="p">}</span>
<span class="linenos"> 80</span>
<span class="linenos"> 81</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Status</span><span class="w"> </span><span class="n">DoAction</span><span class="p">(</span><span class="k">const</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">ServerCallContext</span><span class="o">&amp;</span><span class="p">,</span>
<span class="linenos"> 82</span><span class="w"> </span><span class="k">const</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">Action</span><span class="o">&amp;</span><span class="w"> </span><span class="n">action</span><span class="p">,</span>
<span class="linenos"> 83</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">ResultStream</span><span class="o">&gt;*</span><span class="w"> </span><span class="n">result</span><span class="p">)</span><span class="w"> </span><span class="k">override</span><span class="w"> </span><span class="p">{</span>
<span class="linenos"> 84</span><span class="w"> </span><span class="k">if</span><span class="w"> </span><span class="p">(</span><span class="n">action</span><span class="p">.</span><span class="n">type</span><span class="w"> </span><span class="o">==</span><span class="w"> </span><span class="n">kActionDropDataset</span><span class="p">.</span><span class="n">type</span><span class="p">)</span><span class="w"> </span><span class="p">{</span>
<span class="linenos"> 85</span><span class="w"> </span><span class="o">*</span><span class="n">result</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">ResultStream</span><span class="o">&gt;</span><span class="p">(</span>
<span class="linenos"> 86</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">SimpleResultStream</span><span class="p">({}));</span>
<span class="linenos"> 87</span><span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="n">DoActionDropDataset</span><span class="p">(</span><span class="n">action</span><span class="p">.</span><span class="n">body</span><span class="o">-&gt;</span><span class="n">ToString</span><span class="p">());</span>
<span class="linenos"> 88</span><span class="w"> </span><span class="p">}</span>
<span class="linenos"> 89</span><span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Status</span><span class="o">::</span><span class="n">NotImplemented</span><span class="p">(</span><span class="s">&quot;Unknown action type: &quot;</span><span class="p">,</span><span class="w"> </span><span class="n">action</span><span class="p">.</span><span class="n">type</span><span class="p">);</span>
<span class="linenos"> 90</span><span class="w"> </span><span class="p">}</span>
<span class="linenos"> 91</span>
<span class="linenos"> 92</span><span class="w"> </span><span class="k">private</span><span class="o">:</span>
<span class="linenos"> 93</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Result</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightInfo</span><span class="o">&gt;</span><span class="w"> </span><span class="n">MakeFlightInfo</span><span class="p">(</span>
<span class="linenos"> 94</span><span class="w"> </span><span class="k">const</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">fs</span><span class="o">::</span><span class="n">FileInfo</span><span class="o">&amp;</span><span class="w"> </span><span class="n">file_info</span><span class="p">)</span><span class="w"> </span><span class="p">{</span>
<span class="linenos"> 95</span><span class="w"> </span><span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="k">auto</span><span class="w"> </span><span class="n">input</span><span class="p">,</span><span class="w"> </span><span class="n">root_</span><span class="o">-&gt;</span><span class="n">OpenInputFile</span><span class="p">(</span><span class="n">file_info</span><span class="p">));</span>
<span class="linenos"> 96</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">parquet</span><span class="o">::</span><span class="n">arrow</span><span class="o">::</span><span class="n">FileReader</span><span class="o">&gt;</span><span class="w"> </span><span class="n">reader</span><span class="p">;</span>
<span class="linenos"> 97</span><span class="w"> </span><span class="n">ARROW_RETURN_NOT_OK</span><span class="p">(</span><span class="n">parquet</span><span class="o">::</span><span class="n">arrow</span><span class="o">::</span><span class="n">OpenFile</span><span class="p">(</span><span class="n">std</span><span class="o">::</span><span class="n">move</span><span class="p">(</span><span class="n">input</span><span class="p">),</span>
<span class="linenos"> 98</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">default_memory_pool</span><span class="p">(),</span><span class="w"> </span><span class="o">&amp;</span><span class="n">reader</span><span class="p">));</span>
<span class="linenos"> 99</span>
<span class="linenos">100</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">shared_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">Schema</span><span class="o">&gt;</span><span class="w"> </span><span class="n">schema</span><span class="p">;</span>
<span class="linenos">101</span><span class="w"> </span><span class="n">ARROW_RETURN_NOT_OK</span><span class="p">(</span><span class="n">reader</span><span class="o">-&gt;</span><span class="n">GetSchema</span><span class="p">(</span><span class="o">&amp;</span><span class="n">schema</span><span class="p">));</span>
<span class="linenos">102</span>
<span class="linenos">103</span><span class="w"> </span><span class="k">auto</span><span class="w"> </span><span class="n">descriptor</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightDescriptor</span><span class="o">::</span><span class="n">Path</span><span class="p">({</span><span class="n">file_info</span><span class="p">.</span><span class="n">base_name</span><span class="p">()});</span>
<span class="linenos">104</span>
<span class="linenos">105</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightEndpoint</span><span class="w"> </span><span class="n">endpoint</span><span class="p">;</span>
<span class="linenos">106</span><span class="w"> </span><span class="n">endpoint</span><span class="p">.</span><span class="n">ticket</span><span class="p">.</span><span class="n">ticket</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">file_info</span><span class="p">.</span><span class="n">base_name</span><span class="p">();</span>
<span class="linenos">107</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">Location</span><span class="w"> </span><span class="n">location</span><span class="p">;</span>
<span class="linenos">108</span><span class="w"> </span><span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">location</span><span class="p">,</span>
<span class="linenos">109</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">Location</span><span class="o">::</span><span class="n">ForGrpcTcp</span><span class="p">(</span><span class="s">&quot;localhost&quot;</span><span class="p">,</span><span class="w"> </span><span class="n">port</span><span class="p">()));</span>
<span class="linenos">110</span><span class="w"> </span><span class="n">endpoint</span><span class="p">.</span><span class="n">locations</span><span class="p">.</span><span class="n">push_back</span><span class="p">(</span><span class="n">location</span><span class="p">);</span>
<span class="linenos">111</span>
<span class="linenos">112</span><span class="w"> </span><span class="kt">int64_t</span><span class="w"> </span><span class="n">total_records</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">reader</span><span class="o">-&gt;</span><span class="n">parquet_reader</span><span class="p">()</span><span class="o">-&gt;</span><span class="n">metadata</span><span class="p">()</span><span class="o">-&gt;</span><span class="n">num_rows</span><span class="p">();</span>
<span class="linenos">113</span><span class="w"> </span><span class="kt">int64_t</span><span class="w"> </span><span class="n">total_bytes</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">file_info</span><span class="p">.</span><span class="n">size</span><span class="p">();</span>
<span class="linenos">114</span>
<span class="linenos">115</span><span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightInfo</span><span class="o">::</span><span class="n">Make</span><span class="p">(</span><span class="o">*</span><span class="n">schema</span><span class="p">,</span><span class="w"> </span><span class="n">descriptor</span><span class="p">,</span><span class="w"> </span><span class="p">{</span><span class="n">endpoint</span><span class="p">},</span><span class="w"> </span><span class="n">total_records</span><span class="p">,</span>
<span class="linenos">116</span><span class="w"> </span><span class="n">total_bytes</span><span class="p">);</span>
<span class="linenos">117</span><span class="w"> </span><span class="p">}</span>
<span class="linenos">118</span>
<span class="linenos">119</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Result</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">fs</span><span class="o">::</span><span class="n">FileInfo</span><span class="o">&gt;</span><span class="w"> </span><span class="n">FileInfoFromDescriptor</span><span class="p">(</span>
<span class="linenos">120</span><span class="w"> </span><span class="k">const</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightDescriptor</span><span class="o">&amp;</span><span class="w"> </span><span class="n">descriptor</span><span class="p">)</span><span class="w"> </span><span class="p">{</span>
<span class="linenos">121</span><span class="w"> </span><span class="k">if</span><span class="w"> </span><span class="p">(</span><span class="n">descriptor</span><span class="p">.</span><span class="n">type</span><span class="w"> </span><span class="o">!=</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightDescriptor</span><span class="o">::</span><span class="n">PATH</span><span class="p">)</span><span class="w"> </span><span class="p">{</span>
<span class="linenos">122</span><span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Status</span><span class="o">::</span><span class="n">Invalid</span><span class="p">(</span><span class="s">&quot;Must provide PATH-type FlightDescriptor&quot;</span><span class="p">);</span>
<span class="linenos">123</span><span class="w"> </span><span class="p">}</span><span class="w"> </span><span class="k">else</span><span class="w"> </span><span class="k">if</span><span class="w"> </span><span class="p">(</span><span class="n">descriptor</span><span class="p">.</span><span class="n">path</span><span class="p">.</span><span class="n">size</span><span class="p">()</span><span class="w"> </span><span class="o">!=</span><span class="w"> </span><span class="mi">1</span><span class="p">)</span><span class="w"> </span><span class="p">{</span>
<span class="linenos">124</span><span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Status</span><span class="o">::</span><span class="n">Invalid</span><span class="p">(</span>
<span class="linenos">125</span><span class="w"> </span><span class="s">&quot;Must provide PATH-type FlightDescriptor with one path component&quot;</span><span class="p">);</span>
<span class="linenos">126</span><span class="w"> </span><span class="p">}</span>
<span class="linenos">127</span><span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="n">root_</span><span class="o">-&gt;</span><span class="n">GetFileInfo</span><span class="p">(</span><span class="n">descriptor</span><span class="p">.</span><span class="n">path</span><span class="p">[</span><span class="mi">0</span><span class="p">]);</span>
<span class="linenos">128</span><span class="w"> </span><span class="p">}</span>
<span class="linenos">129</span>
<span class="linenos">130</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Status</span><span class="w"> </span><span class="n">DoActionDropDataset</span><span class="p">(</span><span class="k">const</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">string</span><span class="o">&amp;</span><span class="w"> </span><span class="n">key</span><span class="p">)</span><span class="w"> </span><span class="p">{</span>
<span class="linenos">131</span><span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="n">root_</span><span class="o">-&gt;</span><span class="n">DeleteFile</span><span class="p">(</span><span class="n">key</span><span class="p">);</span>
<span class="linenos">132</span><span class="w"> </span><span class="p">}</span>
<span class="linenos">133</span>
<span class="linenos">134</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">shared_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">fs</span><span class="o">::</span><span class="n">FileSystem</span><span class="o">&gt;</span><span class="w"> </span><span class="n">root_</span><span class="p">;</span>
<span class="linenos">135</span><span class="p">};</span><span class="w"> </span><span class="c1">// end ParquetStorageService</span>
</pre></div>
</div>
</div>
<p>First, we’ll start our server:</p>
<div class="highlight-cpp notranslate"><div class="highlight"><pre><span></span><span class="k">auto</span><span class="w"> </span><span class="n">fs</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">make_shared</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">fs</span><span class="o">::</span><span class="n">LocalFileSystem</span><span class="o">&gt;</span><span class="p">();</span>
<span class="n">ARROW_RETURN_NOT_OK</span><span class="p">(</span><span class="n">fs</span><span class="o">-&gt;</span><span class="n">CreateDir</span><span class="p">(</span><span class="s">&quot;./flight_datasets/&quot;</span><span class="p">));</span>
<span class="n">ARROW_RETURN_NOT_OK</span><span class="p">(</span><span class="n">fs</span><span class="o">-&gt;</span><span class="n">DeleteDirContents</span><span class="p">(</span><span class="s">&quot;./flight_datasets/&quot;</span><span class="p">));</span>
<span class="k">auto</span><span class="w"> </span><span class="n">root</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">make_shared</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">fs</span><span class="o">::</span><span class="n">SubTreeFileSystem</span><span class="o">&gt;</span><span class="p">(</span><span class="s">&quot;./flight_datasets/&quot;</span><span class="p">,</span><span class="w"> </span><span class="n">fs</span><span class="p">);</span>
<span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">Location</span><span class="w"> </span><span class="n">server_location</span><span class="p">;</span>
<span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">server_location</span><span class="p">,</span>
<span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">Location</span><span class="o">::</span><span class="n">ForGrpcTcp</span><span class="p">(</span><span class="s">&quot;0.0.0.0&quot;</span><span class="p">,</span><span class="w"> </span><span class="mi">0</span><span class="p">));</span>
<span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightServerOptions</span><span class="w"> </span><span class="nf">options</span><span class="p">(</span><span class="n">server_location</span><span class="p">);</span>
<span class="k">auto</span><span class="w"> </span><span class="n">server</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightServerBase</span><span class="o">&gt;</span><span class="p">(</span>
<span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">ParquetStorageService</span><span class="p">(</span><span class="n">std</span><span class="o">::</span><span class="n">move</span><span class="p">(</span><span class="n">root</span><span class="p">)));</span>
<span class="n">ARROW_RETURN_NOT_OK</span><span class="p">(</span><span class="n">server</span><span class="o">-&gt;</span><span class="n">Init</span><span class="p">(</span><span class="n">options</span><span class="p">));</span>
<span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="s">&quot;Listening on port &quot;</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">server</span><span class="o">-&gt;</span><span class="n">port</span><span class="p">()</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">endl</span><span class="p">;</span>
</pre></div>
</div>
<div class="literal-block-wrapper docutils container" id="id2">
<div class="code-block-caption"><span class="caption-text">Code Output</span><a class="headerlink" href="#id2" title="Link to this code"></a></div>
<div class="highlight-default notranslate"><div class="highlight"><pre><span></span><span class="n">Listening</span> <span class="n">on</span> <span class="n">port</span> <span class="mi">37117</span>
</pre></div>
</div>
</div>
<p>We can then create a client and connect to the server:</p>
<div class="highlight-cpp notranslate"><div class="highlight"><pre><span></span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">Location</span><span class="w"> </span><span class="n">location</span><span class="p">;</span>
<span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">location</span><span class="p">,</span>
<span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">Location</span><span class="o">::</span><span class="n">ForGrpcTcp</span><span class="p">(</span><span class="s">&quot;localhost&quot;</span><span class="p">,</span><span class="w"> </span><span class="n">server</span><span class="o">-&gt;</span><span class="n">port</span><span class="p">()));</span>
<span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightClient</span><span class="o">&gt;</span><span class="w"> </span><span class="n">client</span><span class="p">;</span>
<span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">client</span><span class="p">,</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightClient</span><span class="o">::</span><span class="n">Connect</span><span class="p">(</span><span class="n">location</span><span class="p">));</span>
<span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="s">&quot;Connected to &quot;</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">location</span><span class="p">.</span><span class="n">ToString</span><span class="p">()</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">endl</span><span class="p">;</span>
</pre></div>
</div>
<div class="literal-block-wrapper docutils container" id="id3">
<div class="code-block-caption"><span class="caption-text">Code Output</span><a class="headerlink" href="#id3" title="Link to this code"></a></div>
<div class="highlight-default notranslate"><div class="highlight"><pre><span></span><span class="n">Connected</span> <span class="n">to</span> <span class="n">grpc</span><span class="o">+</span><span class="n">tcp</span><span class="p">:</span><span class="o">//</span><span class="n">localhost</span><span class="p">:</span><span class="mi">37117</span>
</pre></div>
</div>
</div>
<p>First, we’ll create and upload a table, which will get stored in a
Parquet file by the server.</p>
<div class="highlight-cpp notranslate"><div class="highlight"><pre><span></span><span class="c1">// Open example data file to upload</span>
<span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">std</span><span class="o">::</span><span class="n">string</span><span class="w"> </span><span class="n">airquality_path</span><span class="p">,</span>
<span class="w"> </span><span class="n">FindTestDataFile</span><span class="p">(</span><span class="s">&quot;airquality.parquet&quot;</span><span class="p">));</span>
<span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">std</span><span class="o">::</span><span class="n">shared_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">io</span><span class="o">::</span><span class="n">RandomAccessFile</span><span class="o">&gt;</span><span class="w"> </span><span class="n">input</span><span class="p">,</span>
<span class="w"> </span><span class="n">fs</span><span class="o">-&gt;</span><span class="n">OpenInputFile</span><span class="p">(</span><span class="n">airquality_path</span><span class="p">));</span>
<span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">parquet</span><span class="o">::</span><span class="n">arrow</span><span class="o">::</span><span class="n">FileReader</span><span class="o">&gt;</span><span class="w"> </span><span class="n">reader</span><span class="p">;</span>
<span class="n">ARROW_RETURN_NOT_OK</span><span class="p">(</span>
<span class="w"> </span><span class="n">parquet</span><span class="o">::</span><span class="n">arrow</span><span class="o">::</span><span class="n">OpenFile</span><span class="p">(</span><span class="n">std</span><span class="o">::</span><span class="n">move</span><span class="p">(</span><span class="n">input</span><span class="p">),</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">default_memory_pool</span><span class="p">(),</span><span class="w"> </span><span class="o">&amp;</span><span class="n">reader</span><span class="p">));</span>
<span class="k">auto</span><span class="w"> </span><span class="n">descriptor</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightDescriptor</span><span class="o">::</span><span class="n">Path</span><span class="p">({</span><span class="s">&quot;airquality.parquet&quot;</span><span class="p">});</span>
<span class="n">std</span><span class="o">::</span><span class="n">shared_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">Schema</span><span class="o">&gt;</span><span class="w"> </span><span class="n">schema</span><span class="p">;</span>
<span class="n">ARROW_RETURN_NOT_OK</span><span class="p">(</span><span class="n">reader</span><span class="o">-&gt;</span><span class="n">GetSchema</span><span class="p">(</span><span class="o">&amp;</span><span class="n">schema</span><span class="p">));</span>
<span class="c1">// Start the RPC call</span>
<span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightStreamWriter</span><span class="o">&gt;</span><span class="w"> </span><span class="n">writer</span><span class="p">;</span>
<span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightMetadataReader</span><span class="o">&gt;</span><span class="w"> </span><span class="n">metadata_reader</span><span class="p">;</span>
<span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="k">auto</span><span class="w"> </span><span class="n">put_stream</span><span class="p">,</span><span class="w"> </span><span class="n">client</span><span class="o">-&gt;</span><span class="n">DoPut</span><span class="p">(</span><span class="n">descriptor</span><span class="p">,</span><span class="w"> </span><span class="n">schema</span><span class="p">));</span>
<span class="n">writer</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">move</span><span class="p">(</span><span class="n">put_stream</span><span class="p">.</span><span class="n">writer</span><span class="p">);</span>
<span class="n">metadata_reader</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">move</span><span class="p">(</span><span class="n">put_stream</span><span class="p">.</span><span class="n">reader</span><span class="p">);</span>
<span class="c1">// Upload data</span>
<span class="n">std</span><span class="o">::</span><span class="n">shared_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">RecordBatchReader</span><span class="o">&gt;</span><span class="w"> </span><span class="n">batch_reader</span><span class="p">;</span>
<span class="n">std</span><span class="o">::</span><span class="n">vector</span><span class="o">&lt;</span><span class="kt">int</span><span class="o">&gt;</span><span class="w"> </span><span class="n">row_groups</span><span class="p">(</span><span class="n">reader</span><span class="o">-&gt;</span><span class="n">num_row_groups</span><span class="p">());</span>
<span class="n">std</span><span class="o">::</span><span class="n">iota</span><span class="p">(</span><span class="n">row_groups</span><span class="p">.</span><span class="n">begin</span><span class="p">(),</span><span class="w"> </span><span class="n">row_groups</span><span class="p">.</span><span class="n">end</span><span class="p">(),</span><span class="w"> </span><span class="mi">0</span><span class="p">);</span>
<span class="n">ARROW_RETURN_NOT_OK</span><span class="p">(</span><span class="n">reader</span><span class="o">-&gt;</span><span class="n">GetRecordBatchReader</span><span class="p">(</span><span class="n">row_groups</span><span class="p">,</span><span class="w"> </span><span class="o">&amp;</span><span class="n">batch_reader</span><span class="p">));</span>
<span class="kt">int64_t</span><span class="w"> </span><span class="n">batches</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="mi">0</span><span class="p">;</span>
<span class="k">while</span><span class="w"> </span><span class="p">(</span><span class="nb">true</span><span class="p">)</span><span class="w"> </span><span class="p">{</span>
<span class="w"> </span><span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="k">auto</span><span class="w"> </span><span class="n">batch</span><span class="p">,</span><span class="w"> </span><span class="n">batch_reader</span><span class="o">-&gt;</span><span class="n">Next</span><span class="p">());</span>
<span class="w"> </span><span class="k">if</span><span class="w"> </span><span class="p">(</span><span class="o">!</span><span class="n">batch</span><span class="p">)</span><span class="w"> </span><span class="k">break</span><span class="p">;</span>
<span class="w"> </span><span class="n">ARROW_RETURN_NOT_OK</span><span class="p">(</span><span class="n">writer</span><span class="o">-&gt;</span><span class="n">WriteRecordBatch</span><span class="p">(</span><span class="o">*</span><span class="n">batch</span><span class="p">));</span>
<span class="w"> </span><span class="n">batches</span><span class="o">++</span><span class="p">;</span>
<span class="p">}</span>
<span class="n">ARROW_RETURN_NOT_OK</span><span class="p">(</span><span class="n">writer</span><span class="o">-&gt;</span><span class="n">Close</span><span class="p">());</span>
<span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="s">&quot;Wrote &quot;</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">batches</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="s">&quot; batches&quot;</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">endl</span><span class="p">;</span>
</pre></div>
</div>
<div class="literal-block-wrapper docutils container" id="id4">
<div class="code-block-caption"><span class="caption-text">Code Output</span><a class="headerlink" href="#id4" title="Link to this code"></a></div>
<div class="highlight-default notranslate"><div class="highlight"><pre><span></span><span class="n">Wrote</span> <span class="mi">1</span> <span class="n">batches</span>
</pre></div>
</div>
</div>
<p>Once we do so, we can retrieve the metadata for that dataset:</p>
<div class="highlight-cpp notranslate"><div class="highlight"><pre><span></span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightInfo</span><span class="o">&gt;</span><span class="w"> </span><span class="n">flight_info</span><span class="p">;</span>
<span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">flight_info</span><span class="p">,</span><span class="w"> </span><span class="n">client</span><span class="o">-&gt;</span><span class="n">GetFlightInfo</span><span class="p">(</span><span class="n">descriptor</span><span class="p">));</span>
<span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">flight_info</span><span class="o">-&gt;</span><span class="n">descriptor</span><span class="p">().</span><span class="n">ToString</span><span class="p">()</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">endl</span><span class="p">;</span>
<span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="s">&quot;=== Schema ===&quot;</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">endl</span><span class="p">;</span>
<span class="n">std</span><span class="o">::</span><span class="n">shared_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">Schema</span><span class="o">&gt;</span><span class="w"> </span><span class="n">info_schema</span><span class="p">;</span>
<span class="n">arrow</span><span class="o">::</span><span class="n">ipc</span><span class="o">::</span><span class="n">DictionaryMemo</span><span class="w"> </span><span class="n">dictionary_memo</span><span class="p">;</span>
<span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">info_schema</span><span class="p">,</span><span class="w"> </span><span class="n">flight_info</span><span class="o">-&gt;</span><span class="n">GetSchema</span><span class="p">(</span><span class="o">&amp;</span><span class="n">dictionary_memo</span><span class="p">));</span>
<span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">info_schema</span><span class="o">-&gt;</span><span class="n">ToString</span><span class="p">()</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">endl</span><span class="p">;</span>
<span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="s">&quot;==============&quot;</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">endl</span><span class="p">;</span>
</pre></div>
</div>
<div class="literal-block-wrapper docutils container" id="id5">
<div class="code-block-caption"><span class="caption-text">Code Output</span><a class="headerlink" href="#id5" title="Link to this code"></a></div>
<div class="highlight-default notranslate"><div class="highlight"><pre><span></span><span class="o">&lt;</span><span class="n">FlightDescriptor</span> <span class="n">path</span><span class="o">=</span><span class="s1">&#39;airquality.parquet&#39;</span><span class="o">&gt;</span>
<span class="o">===</span> <span class="n">Schema</span> <span class="o">===</span>
<span class="n">Ozone</span><span class="p">:</span> <span class="n">int32</span>
<span class="n">Solar</span><span class="o">.</span><span class="n">R</span><span class="p">:</span> <span class="n">int32</span>
<span class="n">Wind</span><span class="p">:</span> <span class="n">double</span>
<span class="n">Temp</span><span class="p">:</span> <span class="n">int32</span>
<span class="n">Month</span><span class="p">:</span> <span class="n">int32</span>
<span class="n">Day</span><span class="p">:</span> <span class="n">int32</span>
<span class="o">==============</span>
</pre></div>
</div>
</div>
<p>And get the data back:</p>
<div class="highlight-cpp notranslate"><div class="highlight"><pre><span></span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightStreamReader</span><span class="o">&gt;</span><span class="w"> </span><span class="n">stream</span><span class="p">;</span>
<span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">stream</span><span class="p">,</span><span class="w"> </span><span class="n">client</span><span class="o">-&gt;</span><span class="n">DoGet</span><span class="p">(</span><span class="n">flight_info</span><span class="o">-&gt;</span><span class="n">endpoints</span><span class="p">()[</span><span class="mi">0</span><span class="p">].</span><span class="n">ticket</span><span class="p">));</span>
<span class="n">std</span><span class="o">::</span><span class="n">shared_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">Table</span><span class="o">&gt;</span><span class="w"> </span><span class="n">table</span><span class="p">;</span>
<span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">table</span><span class="p">,</span><span class="w"> </span><span class="n">stream</span><span class="o">-&gt;</span><span class="n">ToTable</span><span class="p">());</span>
<span class="n">arrow</span><span class="o">::</span><span class="n">PrettyPrintOptions</span><span class="w"> </span><span class="nf">print_options</span><span class="p">(</span><span class="cm">/*indent=*/</span><span class="mi">0</span><span class="p">,</span><span class="w"> </span><span class="cm">/*window=*/</span><span class="mi">2</span><span class="p">);</span>
<span class="n">ARROW_RETURN_NOT_OK</span><span class="p">(</span><span class="n">arrow</span><span class="o">::</span><span class="n">PrettyPrint</span><span class="p">(</span><span class="o">*</span><span class="n">table</span><span class="p">,</span><span class="w"> </span><span class="n">print_options</span><span class="p">,</span><span class="w"> </span><span class="o">&amp;</span><span class="n">rout</span><span class="p">));</span>
</pre></div>
</div>
<div class="literal-block-wrapper docutils container" id="id6">
<div class="code-block-caption"><span class="caption-text">Code Output</span><a class="headerlink" href="#id6" title="Link to this code"></a></div>
<div class="highlight-default notranslate"><div class="highlight"><pre><span></span><span class="n">Ozone</span><span class="p">:</span> <span class="n">int32</span>
<span class="n">Solar</span><span class="o">.</span><span class="n">R</span><span class="p">:</span> <span class="n">int32</span>
<span class="n">Wind</span><span class="p">:</span> <span class="n">double</span>
<span class="n">Temp</span><span class="p">:</span> <span class="n">int32</span>
<span class="n">Month</span><span class="p">:</span> <span class="n">int32</span>
<span class="n">Day</span><span class="p">:</span> <span class="n">int32</span>
<span class="o">----</span>
<span class="n">Ozone</span><span class="p">:</span>
<span class="p">[</span>
<span class="p">[</span>
<span class="mi">41</span><span class="p">,</span>
<span class="mi">36</span><span class="p">,</span>
<span class="o">...</span>
<span class="mi">18</span><span class="p">,</span>
<span class="mi">20</span>
<span class="p">]</span>
<span class="p">]</span>
<span class="n">Solar</span><span class="o">.</span><span class="n">R</span><span class="p">:</span>
<span class="p">[</span>
<span class="p">[</span>
<span class="mi">190</span><span class="p">,</span>
<span class="mi">118</span><span class="p">,</span>
<span class="o">...</span>
<span class="mi">131</span><span class="p">,</span>
<span class="mi">223</span>
<span class="p">]</span>
<span class="p">]</span>
<span class="n">Wind</span><span class="p">:</span>
<span class="p">[</span>
<span class="p">[</span>
<span class="mf">7.4</span><span class="p">,</span>
<span class="mi">8</span><span class="p">,</span>
<span class="o">...</span>
<span class="mi">8</span><span class="p">,</span>
<span class="mf">11.5</span>
<span class="p">]</span>
<span class="p">]</span>
<span class="n">Temp</span><span class="p">:</span>
<span class="p">[</span>
<span class="p">[</span>
<span class="mi">67</span><span class="p">,</span>
<span class="mi">72</span><span class="p">,</span>
<span class="o">...</span>
<span class="mi">76</span><span class="p">,</span>
<span class="mi">68</span>
<span class="p">]</span>
<span class="p">]</span>
<span class="n">Month</span><span class="p">:</span>
<span class="p">[</span>
<span class="p">[</span>
<span class="mi">5</span><span class="p">,</span>
<span class="mi">5</span><span class="p">,</span>
<span class="o">...</span>
<span class="mi">9</span><span class="p">,</span>
<span class="mi">9</span>
<span class="p">]</span>
<span class="p">]</span>
<span class="n">Day</span><span class="p">:</span>
<span class="p">[</span>
<span class="p">[</span>
<span class="mi">1</span><span class="p">,</span>
<span class="mi">2</span><span class="p">,</span>
<span class="o">...</span>
<span class="mi">29</span><span class="p">,</span>
<span class="mi">30</span>
<span class="p">]</span>
<span class="p">]</span>
</pre></div>
</div>
</div>
<p>Then, we’ll delete the dataset:</p>
<div class="highlight-cpp notranslate"><div class="highlight"><pre><span></span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">Action</span><span class="w"> </span><span class="n">action</span><span class="p">{</span><span class="s">&quot;drop_dataset&quot;</span><span class="p">,</span>
<span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Buffer</span><span class="o">::</span><span class="n">FromString</span><span class="p">(</span><span class="s">&quot;airquality.parquet&quot;</span><span class="p">)};</span>
<span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">ResultStream</span><span class="o">&gt;</span><span class="w"> </span><span class="n">results</span><span class="p">;</span>
<span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">results</span><span class="p">,</span><span class="w"> </span><span class="n">client</span><span class="o">-&gt;</span><span class="n">DoAction</span><span class="p">(</span><span class="n">action</span><span class="p">));</span>
<span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="s">&quot;Deleted dataset&quot;</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">endl</span><span class="p">;</span>
</pre></div>
</div>
<div class="literal-block-wrapper docutils container" id="id7">
<div class="code-block-caption"><span class="caption-text">Code Output</span><a class="headerlink" href="#id7" title="Link to this code"></a></div>
<div class="highlight-default notranslate"><div class="highlight"><pre><span></span><span class="n">Deleted</span> <span class="n">dataset</span>
</pre></div>
</div>
</div>
<p>And confirm that it’s been deleted:</p>
<div class="highlight-cpp notranslate"><div class="highlight"><pre><span></span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightListing</span><span class="o">&gt;</span><span class="w"> </span><span class="n">listing</span><span class="p">;</span>
<span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">listing</span><span class="p">,</span><span class="w"> </span><span class="n">client</span><span class="o">-&gt;</span><span class="n">ListFlights</span><span class="p">());</span>
<span class="k">while</span><span class="w"> </span><span class="p">(</span><span class="nb">true</span><span class="p">)</span><span class="w"> </span><span class="p">{</span>
<span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightInfo</span><span class="o">&gt;</span><span class="w"> </span><span class="n">flight_info</span><span class="p">;</span>
<span class="w"> </span><span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">flight_info</span><span class="p">,</span><span class="w"> </span><span class="n">listing</span><span class="o">-&gt;</span><span class="n">Next</span><span class="p">());</span>
<span class="w"> </span><span class="k">if</span><span class="w"> </span><span class="p">(</span><span class="o">!</span><span class="n">flight_info</span><span class="p">)</span><span class="w"> </span><span class="k">break</span><span class="p">;</span>
<span class="w"> </span><span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">flight_info</span><span class="o">-&gt;</span><span class="n">descriptor</span><span class="p">().</span><span class="n">ToString</span><span class="p">()</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">endl</span><span class="p">;</span>
<span class="w"> </span><span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="s">&quot;=== Schema ===&quot;</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">endl</span><span class="p">;</span>
<span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">shared_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">Schema</span><span class="o">&gt;</span><span class="w"> </span><span class="n">info_schema</span><span class="p">;</span>
<span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">ipc</span><span class="o">::</span><span class="n">DictionaryMemo</span><span class="w"> </span><span class="n">dictionary_memo</span><span class="p">;</span>
<span class="w"> </span><span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">info_schema</span><span class="p">,</span><span class="w"> </span><span class="n">flight_info</span><span class="o">-&gt;</span><span class="n">GetSchema</span><span class="p">(</span><span class="o">&amp;</span><span class="n">dictionary_memo</span><span class="p">));</span>
<span class="w"> </span><span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">info_schema</span><span class="o">-&gt;</span><span class="n">ToString</span><span class="p">()</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">endl</span><span class="p">;</span>
<span class="w"> </span><span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="s">&quot;==============&quot;</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">endl</span><span class="p">;</span>
<span class="p">}</span>
<span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="s">&quot;End of listing&quot;</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">endl</span><span class="p">;</span>
</pre></div>
</div>
<div class="literal-block-wrapper docutils container" id="id8">
<div class="code-block-caption"><span class="caption-text">Code Output</span><a class="headerlink" href="#id8" title="Link to this code"></a></div>
<div class="highlight-default notranslate"><div class="highlight"><pre><span></span><span class="n">End</span> <span class="n">of</span> <span class="n">listing</span>
</pre></div>
</div>
</div>
<p>Finally, we’ll stop our server:</p>
<div class="highlight-cpp notranslate"><div class="highlight"><pre><span></span><span class="n">ARROW_RETURN_NOT_OK</span><span class="p">(</span><span class="n">server</span><span class="o">-&gt;</span><span class="n">Shutdown</span><span class="p">());</span>
<span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="s">&quot;Server shut down successfully&quot;</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">endl</span><span class="p">;</span>
</pre></div>
</div>
<div class="literal-block-wrapper docutils container" id="id9">
<div class="code-block-caption"><span class="caption-text">Code Output</span><a class="headerlink" href="#id9" title="Link to this code"></a></div>
<div class="highlight-default notranslate"><div class="highlight"><pre><span></span><span class="n">Server</span> <span class="n">shut</span> <span class="n">down</span> <span class="n">successfully</span>
</pre></div>
</div>
</div>
</section>
<section id="setting-grpc-client-options">
<h2><a class="toc-backref" href="#id17" role="doc-backlink">Setting gRPC client options</a><a class="headerlink" href="#setting-grpc-client-options" title="Link to this heading"></a></h2>
<p>Options for gRPC clients can be passed in using the <code class="docutils literal notranslate"><span class="pre">generic_options</span></code> field of
<a class="reference external" href="https://arrow.apache.org/docs/cpp/api/flight.html#_CPPv4N5arrow6flight19FlightClientOptionsE" title="(in Apache Arrow v15.0.1)"><code class="xref cpp cpp-class docutils literal notranslate"><span class="pre">arrow::flight::FlightClientOptions</span></code></a>. There is a list of available
client options in the <a class="reference external" href="https://grpc.github.io/grpc/cpp/group__grpc__arg__keys.html">gRPC API documentation</a>.</p>
<p>For example, you can change the maximum message length sent with:</p>
<div class="highlight-cpp notranslate"><div class="highlight"><pre><span></span><span class="k">auto</span><span class="w"> </span><span class="n">client_options</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightClientOptions</span><span class="o">::</span><span class="n">Defaults</span><span class="p">();</span>
<span class="c1">// Set a very low limit at the gRPC layer to fail all calls</span>
<span class="n">client_options</span><span class="p">.</span><span class="n">generic_options</span><span class="p">.</span><span class="n">emplace_back</span><span class="p">(</span><span class="n">GRPC_ARG_MAX_SEND_MESSAGE_LENGTH</span><span class="p">,</span><span class="w"> </span><span class="mi">2</span><span class="p">);</span>
<span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">Location</span><span class="w"> </span><span class="n">location</span><span class="p">;</span>
<span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">location</span><span class="p">,</span>
<span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">Location</span><span class="o">::</span><span class="n">ForGrpcTcp</span><span class="p">(</span><span class="s">&quot;localhost&quot;</span><span class="p">,</span><span class="w"> </span><span class="n">server</span><span class="o">-&gt;</span><span class="n">port</span><span class="p">()));</span>
<span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightClient</span><span class="o">&gt;</span><span class="w"> </span><span class="n">client</span><span class="p">;</span>
<span class="c1">// pass client_options into Connect()</span>
<span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">client</span><span class="p">,</span>
<span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightClient</span><span class="o">::</span><span class="n">Connect</span><span class="p">(</span><span class="n">location</span><span class="p">,</span><span class="w"> </span><span class="n">client_options</span><span class="p">));</span>
<span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="s">&quot;Connected to &quot;</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">location</span><span class="p">.</span><span class="n">ToString</span><span class="p">()</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">endl</span><span class="p">;</span>
</pre></div>
</div>
<div class="literal-block-wrapper docutils container" id="id10">
<div class="code-block-caption"><span class="caption-text">Code Output</span><a class="headerlink" href="#id10" title="Link to this code"></a></div>
<div class="highlight-default notranslate"><div class="highlight"><pre><span></span><span class="n">Connected</span> <span class="n">to</span> <span class="n">grpc</span><span class="o">+</span><span class="n">tcp</span><span class="p">:</span><span class="o">//</span><span class="n">localhost</span><span class="p">:</span><span class="mi">41083</span>
</pre></div>
</div>
</div>
</section>
<section id="flight-service-with-other-grpc-endpoints">
<h2><a class="toc-backref" href="#id18" role="doc-backlink">Flight Service with other gRPC endpoints</a><a class="headerlink" href="#flight-service-with-other-grpc-endpoints" title="Link to this heading"></a></h2>
<p>If you are using the gRPC backend, you can add other gRPC endpoints to the
Flight server. While Flight clients won’t recognize these endpoints, general
gRPC clients will be able to.</p>
<div class="admonition note">
<p class="admonition-title">Note</p>
<p>If statically linking Arrow Flight, Protobuf and gRPC must also be statically
linked, and the same goes for dynamic linking. Read more at
<a class="reference external" href="https://arrow.apache.org/docs/cpp/build_system.html#a-note-on-linking">https://arrow.apache.org/docs/cpp/build_system.html#a-note-on-linking</a></p>
</div>
<section id="creating-the-server">
<h3><a class="toc-backref" href="#id19" role="doc-backlink">Creating the server</a><a class="headerlink" href="#creating-the-server" title="Link to this heading"></a></h3>
<p>To create a gRPC service, first define a service using protobuf.</p>
<div class="literal-block-wrapper docutils container" id="id11">
<div class="code-block-caption"><span class="caption-text">Hello world protobuf specification</span><a class="headerlink" href="#id11" title="Link to this code"></a></div>
<div class="highlight-protobuf notranslate"><div class="highlight"><pre><span></span><span class="linenos"> 1</span><span class="k">syntax</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s">&quot;proto3&quot;</span><span class="p">;</span>
<span class="linenos"> 2</span>
<span class="linenos"> 3</span><span class="kd">service</span><span class="w"> </span><span class="n">HelloWorldService</span><span class="w"> </span><span class="p">{</span>
<span class="linenos"> 4</span><span class="w"> </span><span class="k">rpc</span><span class="w"> </span><span class="n">SayHello</span><span class="p">(</span><span class="n">HelloRequest</span><span class="p">)</span><span class="w"> </span><span class="k">returns</span><span class="w"> </span><span class="p">(</span><span class="n">HelloResponse</span><span class="p">);</span>
<span class="linenos"> 5</span><span class="p">}</span>
<span class="linenos"> 6</span>
<span class="linenos"> 7</span><span class="kd">message</span><span class="w"> </span><span class="nc">HelloRequest</span><span class="w"> </span><span class="p">{</span>
<span class="linenos"> 8</span><span class="w"> </span><span class="kt">string</span><span class="w"> </span><span class="na">name</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="mi">1</span><span class="p">;</span>
<span class="linenos"> 9</span><span class="p">}</span>
<span class="linenos">10</span>
<span class="linenos">11</span><span class="kd">message</span><span class="w"> </span><span class="nc">HelloResponse</span><span class="w"> </span><span class="p">{</span>
<span class="linenos">12</span><span class="w"> </span><span class="kt">string</span><span class="w"> </span><span class="na">reply</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="mi">1</span><span class="p">;</span>
<span class="linenos">13</span><span class="p">}</span>
</pre></div>
</div>
</div>
<p>Next, you’ll need to compile that to provide the protobuf and gRPC generated
files. See gRPC’s <a class="reference external" href="https://grpc.io/docs/languages/cpp/basics/#generating-client-and-server-code">generating client and server code</a>
docs for details.</p>
<p>Then write an implementation for the gRPC service:</p>
<div class="literal-block-wrapper docutils container" id="id12">
<div class="code-block-caption"><span class="caption-text">Hello world gRPC service implementation</span><a class="headerlink" href="#id12" title="Link to this code"></a></div>
<div class="highlight-cpp notranslate"><div class="highlight"><pre><span></span><span class="linenos"> 1</span><span class="k">class</span><span class="w"> </span><span class="nc">HelloWorldServiceImpl</span><span class="w"> </span><span class="o">:</span><span class="w"> </span><span class="k">public</span><span class="w"> </span><span class="n">HelloWorldService</span><span class="o">::</span><span class="n">Service</span><span class="w"> </span><span class="p">{</span>
<span class="linenos"> 2</span><span class="w"> </span><span class="n">grpc</span><span class="o">::</span><span class="n">Status</span><span class="w"> </span><span class="nf">SayHello</span><span class="p">(</span><span class="n">grpc</span><span class="o">::</span><span class="n">ServerContext</span><span class="o">*</span><span class="p">,</span><span class="w"> </span><span class="k">const</span><span class="w"> </span><span class="n">HelloRequest</span><span class="o">*</span><span class="w"> </span><span class="n">request</span><span class="p">,</span>
<span class="linenos"> 3</span><span class="w"> </span><span class="n">HelloResponse</span><span class="o">*</span><span class="w"> </span><span class="n">reply</span><span class="p">)</span><span class="w"> </span><span class="k">override</span><span class="w"> </span><span class="p">{</span>
<span class="linenos"> 4</span><span class="w"> </span><span class="k">const</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">string</span><span class="o">&amp;</span><span class="w"> </span><span class="n">name</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">request</span><span class="o">-&gt;</span><span class="n">name</span><span class="p">();</span>
<span class="linenos"> 5</span><span class="w"> </span><span class="k">if</span><span class="w"> </span><span class="p">(</span><span class="n">name</span><span class="p">.</span><span class="n">empty</span><span class="p">())</span><span class="w"> </span><span class="p">{</span>
<span class="linenos"> 6</span><span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="n">grpc</span><span class="o">::</span><span class="n">Status</span><span class="p">(</span><span class="n">grpc</span><span class="o">::</span><span class="n">StatusCode</span><span class="o">::</span><span class="n">INVALID_ARGUMENT</span><span class="p">,</span><span class="w"> </span><span class="s">&quot;Must provide a name!&quot;</span><span class="p">);</span>
<span class="linenos"> 7</span><span class="w"> </span><span class="p">}</span>
<span class="linenos"> 8</span><span class="w"> </span><span class="n">reply</span><span class="o">-&gt;</span><span class="n">set_reply</span><span class="p">(</span><span class="s">&quot;Hello, &quot;</span><span class="w"> </span><span class="o">+</span><span class="w"> </span><span class="n">name</span><span class="p">);</span>
<span class="linenos"> 9</span><span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="n">grpc</span><span class="o">::</span><span class="n">Status</span><span class="o">::</span><span class="n">OK</span><span class="p">;</span>
<span class="linenos">10</span><span class="w"> </span><span class="p">}</span>
<span class="linenos">11</span><span class="p">};</span><span class="w"> </span><span class="c1">// end HelloWorldServiceImpl</span>
</pre></div>
</div>
</div>
<p>Finally, use the <code class="docutils literal notranslate"><span class="pre">builder_hook</span></code> hook on <a class="reference external" href="https://arrow.apache.org/docs/cpp/api/flight.html#_CPPv4N5arrow6flight19FlightServerOptionsE" title="(in Apache Arrow v15.0.1)"><code class="xref cpp cpp-class docutils literal notranslate"><span class="pre">arrow::flight::FlightServerOptions</span></code></a>
to register the additional gRPC service.</p>
<div class="highlight-cpp notranslate"><div class="highlight"><pre><span></span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">Location</span><span class="w"> </span><span class="n">server_location</span><span class="p">;</span>
<span class="n">ARROW_ASSIGN_OR_RAISE</span><span class="p">(</span><span class="n">server_location</span><span class="p">,</span>
<span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">Location</span><span class="o">::</span><span class="n">ForGrpcTcp</span><span class="p">(</span><span class="s">&quot;0.0.0.0&quot;</span><span class="p">,</span><span class="w"> </span><span class="mi">5000</span><span class="p">));</span>
<span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightServerOptions</span><span class="w"> </span><span class="nf">options</span><span class="p">(</span><span class="n">server_location</span><span class="p">);</span>
<span class="k">auto</span><span class="w"> </span><span class="n">server</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">unique_ptr</span><span class="o">&lt;</span><span class="n">arrow</span><span class="o">::</span><span class="n">flight</span><span class="o">::</span><span class="n">FlightServerBase</span><span class="o">&gt;</span><span class="p">(</span>
<span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">ParquetStorageService</span><span class="p">(</span><span class="n">std</span><span class="o">::</span><span class="n">move</span><span class="p">(</span><span class="n">root</span><span class="p">)));</span>
<span class="c1">// Create hello world service</span>
<span class="n">HelloWorldServiceImpl</span><span class="w"> </span><span class="n">grpc_service</span><span class="p">;</span>
<span class="c1">// Use builder_hook to register grpc service</span>
<span class="n">options</span><span class="p">.</span><span class="n">builder_hook</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="p">[</span><span class="o">&amp;</span><span class="p">](</span><span class="kt">void</span><span class="o">*</span><span class="w"> </span><span class="n">raw_builder</span><span class="p">)</span><span class="w"> </span><span class="p">{</span>
<span class="w"> </span><span class="k">auto</span><span class="o">*</span><span class="w"> </span><span class="n">builder</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">reinterpret_cast</span><span class="o">&lt;</span><span class="n">grpc</span><span class="o">::</span><span class="n">ServerBuilder</span><span class="o">*&gt;</span><span class="p">(</span><span class="n">raw_builder</span><span class="p">);</span>
<span class="w"> </span><span class="n">builder</span><span class="o">-&gt;</span><span class="n">RegisterService</span><span class="p">(</span><span class="o">&amp;</span><span class="n">grpc_service</span><span class="p">);</span>
<span class="p">};</span>
<span class="n">ARROW_RETURN_NOT_OK</span><span class="p">(</span><span class="n">server</span><span class="o">-&gt;</span><span class="n">Init</span><span class="p">(</span><span class="n">options</span><span class="p">));</span>
<span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="s">&quot;Listening on port &quot;</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">server</span><span class="o">-&gt;</span><span class="n">port</span><span class="p">()</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">std</span><span class="o">::</span><span class="n">endl</span><span class="p">;</span>
</pre></div>
</div>
<div class="literal-block-wrapper docutils container" id="id13">
<div class="code-block-caption"><span class="caption-text">Code Output</span><a class="headerlink" href="#id13" title="Link to this code"></a></div>
<div class="highlight-default notranslate"><div class="highlight"><pre><span></span><span class="n">Listening</span> <span class="n">on</span> <span class="n">port</span> <span class="mi">5000</span>
</pre></div>
</div>
</div>
</section>
<section id="creating-the-client">
<h3><a class="toc-backref" href="#id20" role="doc-backlink">Creating the client</a><a class="headerlink" href="#creating-the-client" title="Link to this heading"></a></h3>
<p>The Flight client implementation doesn’t know about any custom gRPC services,
so to call them you’ll need to create a normal gRPC client. For the Hello World
service, we use the HelloWorldService stub, which is provided by the compiled
gRPC definition.</p>
<div class="highlight-cpp notranslate"><div class="highlight"><pre><span></span><span class="k">auto</span><span class="w"> </span><span class="n">client_channel</span><span class="w"> </span><span class="o">=</span>
<span class="w"> </span><span class="n">grpc</span><span class="o">::</span><span class="n">CreateChannel</span><span class="p">(</span><span class="s">&quot;0.0.0.0:5000&quot;</span><span class="p">,</span><span class="w"> </span><span class="n">grpc</span><span class="o">::</span><span class="n">InsecureChannelCredentials</span><span class="p">());</span>
<span class="k">auto</span><span class="w"> </span><span class="n">stub</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">HelloWorldService</span><span class="o">::</span><span class="n">NewStub</span><span class="p">(</span><span class="n">client_channel</span><span class="p">);</span>
<span class="n">grpc</span><span class="o">::</span><span class="n">ClientContext</span><span class="w"> </span><span class="n">context</span><span class="p">;</span>
<span class="n">HelloRequest</span><span class="w"> </span><span class="n">request</span><span class="p">;</span>
<span class="n">request</span><span class="p">.</span><span class="n">set_name</span><span class="p">(</span><span class="s">&quot;Arrow User&quot;</span><span class="p">);</span>
<span class="n">HelloResponse</span><span class="w"> </span><span class="n">response</span><span class="p">;</span>
<span class="n">grpc</span><span class="o">::</span><span class="n">Status</span><span class="w"> </span><span class="n">status</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">stub</span><span class="o">-&gt;</span><span class="n">SayHello</span><span class="p">(</span><span class="o">&amp;</span><span class="n">context</span><span class="p">,</span><span class="w"> </span><span class="n">request</span><span class="p">,</span><span class="w"> </span><span class="o">&amp;</span><span class="n">response</span><span class="p">);</span>
<span class="k">if</span><span class="w"> </span><span class="p">(</span><span class="o">!</span><span class="n">status</span><span class="p">.</span><span class="n">ok</span><span class="p">())</span><span class="w"> </span><span class="p">{</span>
<span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="n">arrow</span><span class="o">::</span><span class="n">Status</span><span class="o">::</span><span class="n">IOError</span><span class="p">(</span><span class="n">status</span><span class="p">.</span><span class="n">error_message</span><span class="p">());</span>
<span class="p">}</span>
<span class="n">rout</span><span class="w"> </span><span class="o">&lt;&lt;</span><span class="w"> </span><span class="n">response</span><span class="p">.</span><span class="n">reply</span><span class="p">();</span>
</pre></div>
</div>
<div class="literal-block-wrapper docutils container" id="id14">
<div class="code-block-caption"><span class="caption-text">Code Output</span><a class="headerlink" href="#id14" title="Link to this code"></a></div>
<div class="highlight-default notranslate"><div class="highlight"><pre><span></span><span class="n">Hello</span><span class="p">,</span> <span class="n">Arrow</span> <span class="n">User</span>
</pre></div>
</div>
</div>
</section>
</section>
</section>
</div>
</div>
</div>
<div class="sphinxsidebar" role="navigation" aria-label="main navigation">
<div class="sphinxsidebarwrapper">
<p class="logo">
<a href="index.html">
<img class="logo" src="_static/arrow-logo_vertical_black-txt_transparent-bg.svg" alt="Logo" />
</a>
</p>
<p>
<iframe src="https://ghbtns.com/github-btn.html?user=apache&repo=arrow-cookbook&type=none&count=true&size=large&v=2"
allowtransparency="true" frameborder="0" scrolling="0" width="200px" height="35px"></iframe>
</p>
<h3>Navigation</h3>
<p class="caption" role="heading"><span class="caption-text">Contents:</span></p>
<ul class="current">
<li class="toctree-l1"><a class="reference internal" href="basic.html">Working with the C++ Implementation</a></li>
<li class="toctree-l1"><a class="reference internal" href="create.html">Creating Arrow Objects</a></li>
<li class="toctree-l1"><a class="reference internal" href="datasets.html">Reading and Writing Datasets</a></li>
<li class="toctree-l1 current"><a class="current reference internal" href="#">Arrow Flight</a><ul>
<li class="toctree-l2"><a class="reference internal" href="#simple-parquet-storage-service-with-arrow-flight">Simple Parquet storage service with Arrow Flight</a></li>
<li class="toctree-l2"><a class="reference internal" href="#setting-grpc-client-options">Setting gRPC client options</a></li>
<li class="toctree-l2"><a class="reference internal" href="#flight-service-with-other-grpc-endpoints">Flight Service with other gRPC endpoints</a></li>
</ul>
</li>
</ul>
<hr />
<ul>
<li class="toctree-l1"><a href="https://arrow.apache.org/docs/cpp/index.html">User Guide</a></li>
<li class="toctree-l1"><a href="https://arrow.apache.org/docs/cpp/api.html">API Reference</a></li>
</ul>
<div class="relations">
<h3>Related Topics</h3>
<ul>
<li><a href="index.html">Documentation overview</a><ul>
<li>Previous: <a href="datasets.html" title="previous chapter">Reading and Writing Datasets</a></li>
</ul></li>
</ul>
</div>
<div id="searchbox" style="display: none" role="search">
<h3 id="searchlabel">Quick search</h3>
<div class="searchformwrapper">
<form class="search" action="search.html" method="get">
<input type="text" name="q" aria-labelledby="searchlabel" autocomplete="off" autocorrect="off" autocapitalize="off" spellcheck="false"/>
<input type="submit" value="Go" />
</form>
</div>
</div>
<script>document.getElementById('searchbox').style.display = "block"</script>
</div>
</div>
<div class="clearer"></div>
</div>
<div class="footer">
&#169;2022, Apache Software Foundation.
|
Powered by <a href="https://www.sphinx-doc.org/">Sphinx 7.2.6</a>
&amp; <a href="https://alabaster.readthedocs.io">Alabaster 0.7.16</a>
|
<a href="_sources/flight.rst.txt"
rel="nofollow">Page source</a>
</div>
</body>
</html>