| <!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 — Apache Arrow Java 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="next" title="Dataset" href="dataset.html" /> |
| <link rel="prev" title="Reading and writing data" href="io.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="#id1" 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. |
| For more detail about Flight please take a look at <a class="reference internal" href="#arrow-flight-rpc">Arrow Flight RPC</a>.</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="id1">Arrow Flight</a></p> |
| <ul> |
| <li><p><a class="reference internal" href="#simple-key-value-storage-service-with-arrow-flight" id="id2">Simple Key-Value Storage Service with Arrow Flight</a></p> |
| <ul> |
| <li><p><a class="reference internal" href="#flight-client-and-server" id="id3">Flight Client and Server</a></p></li> |
| <li><p><a class="reference internal" href="#start-flight-server" id="id4">Start Flight Server</a></p></li> |
| <li><p><a class="reference internal" href="#connect-to-flight-server" id="id5">Connect to Flight Server</a></p></li> |
| <li><p><a class="reference internal" href="#put-data" id="id6">Put Data</a></p></li> |
| <li><p><a class="reference internal" href="#get-metadata" id="id7">Get Metadata</a></p></li> |
| <li><p><a class="reference internal" href="#get-data" id="id8">Get Data</a></p></li> |
| <li><p><a class="reference internal" href="#delete-data" id="id9">Delete data</a></p></li> |
| <li><p><a class="reference internal" href="#validate-delete-data" id="id10">Validate Delete Data</a></p></li> |
| <li><p><a class="reference internal" href="#stop-flight-server" id="id11">Stop Flight Server</a></p></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| </ul> |
| </nav> |
| <section id="simple-key-value-storage-service-with-arrow-flight"> |
| <h2><a class="toc-backref" href="#id2" role="doc-backlink">Simple Key-Value Storage Service with Arrow Flight</a><a class="headerlink" href="#simple-key-value-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 data, using Flight to handle uploads/requests |
| and data in memory to store the actual data.</p> |
| <section id="flight-client-and-server"> |
| <h3><a class="toc-backref" href="#id3" role="doc-backlink">Flight Client and Server</a><a class="headerlink" href="#flight-client-and-server" title="Link to this heading">¶</a></h3> |
| <div class="highlight-java notranslate"><div class="highlight"><pre><span></span><span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.flight.Action</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.flight.AsyncPutListener</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.flight.CallStatus</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.flight.Criteria</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.flight.FlightClient</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.flight.FlightDescriptor</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.flight.FlightEndpoint</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.flight.FlightInfo</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.flight.FlightServer</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.flight.FlightStream</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.flight.Location</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.flight.NoOpFlightProducer</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.flight.PutResult</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.flight.Result</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.flight.Ticket</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.memory.BufferAllocator</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.memory.RootAllocator</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.util.AutoCloseables</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.vector.VarCharVector</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.vector.VectorLoader</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.vector.VectorSchemaRoot</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.vector.VectorUnloader</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.vector.ipc.message.ArrowRecordBatch</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.vector.types.pojo.ArrowType</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.vector.types.pojo.Field</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.vector.types.pojo.FieldType</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">org.apache.arrow.vector.types.pojo.Schema</span><span class="p">;</span> |
| |
| <span class="kn">import</span><span class="w"> </span><span class="nn">java.io.IOException</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">java.nio.charset.StandardCharsets</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">java.util.ArrayList</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">java.util.Arrays</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">java.util.Collections</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">java.util.Iterator</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">java.util.List</span><span class="p">;</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">java.util.concurrent.ConcurrentHashMap</span><span class="p">;</span> |
| |
| <span class="kd">class</span> <span class="nc">Dataset</span><span class="w"> </span><span class="kd">implements</span><span class="w"> </span><span class="n">AutoCloseable</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="kd">private</span><span class="w"> </span><span class="kd">final</span><span class="w"> </span><span class="n">List</span><span class="o"><</span><span class="n">ArrowRecordBatch</span><span class="o">></span><span class="w"> </span><span class="n">batches</span><span class="p">;</span> |
| <span class="w"> </span><span class="kd">private</span><span class="w"> </span><span class="kd">final</span><span class="w"> </span><span class="n">Schema</span><span class="w"> </span><span class="n">schema</span><span class="p">;</span> |
| <span class="w"> </span><span class="kd">private</span><span class="w"> </span><span class="kd">final</span><span class="w"> </span><span class="kt">long</span><span class="w"> </span><span class="n">rows</span><span class="p">;</span> |
| <span class="w"> </span><span class="kd">public</span><span class="w"> </span><span class="nf">Dataset</span><span class="p">(</span><span class="n">List</span><span class="o"><</span><span class="n">ArrowRecordBatch</span><span class="o">></span><span class="w"> </span><span class="n">batches</span><span class="p">,</span><span class="w"> </span><span class="n">Schema</span><span class="w"> </span><span class="n">schema</span><span class="p">,</span><span class="w"> </span><span class="kt">long</span><span class="w"> </span><span class="n">rows</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="k">this</span><span class="p">.</span><span class="na">batches</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">batches</span><span class="p">;</span> |
| <span class="w"> </span><span class="k">this</span><span class="p">.</span><span class="na">schema</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">schema</span><span class="p">;</span> |
| <span class="w"> </span><span class="k">this</span><span class="p">.</span><span class="na">rows</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">rows</span><span class="p">;</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="kd">public</span><span class="w"> </span><span class="n">List</span><span class="o"><</span><span class="n">ArrowRecordBatch</span><span class="o">></span><span class="w"> </span><span class="nf">getBatches</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">batches</span><span class="p">;</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="kd">public</span><span class="w"> </span><span class="n">Schema</span><span class="w"> </span><span class="nf">getSchema</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">schema</span><span class="p">;</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="kd">public</span><span class="w"> </span><span class="kt">long</span><span class="w"> </span><span class="nf">getRows</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">rows</span><span class="p">;</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="nd">@Override</span> |
| <span class="w"> </span><span class="kd">public</span><span class="w"> </span><span class="kt">void</span><span class="w"> </span><span class="nf">close</span><span class="p">()</span><span class="w"> </span><span class="kd">throws</span><span class="w"> </span><span class="n">Exception</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">AutoCloseables</span><span class="p">.</span><span class="na">close</span><span class="p">(</span><span class="n">batches</span><span class="p">);</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="p">}</span> |
| <span class="kd">class</span> <span class="nc">CookbookProducer</span><span class="w"> </span><span class="kd">extends</span><span class="w"> </span><span class="n">NoOpFlightProducer</span><span class="w"> </span><span class="kd">implements</span><span class="w"> </span><span class="n">AutoCloseable</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="kd">private</span><span class="w"> </span><span class="kd">final</span><span class="w"> </span><span class="n">BufferAllocator</span><span class="w"> </span><span class="n">allocator</span><span class="p">;</span> |
| <span class="w"> </span><span class="kd">private</span><span class="w"> </span><span class="kd">final</span><span class="w"> </span><span class="n">Location</span><span class="w"> </span><span class="n">location</span><span class="p">;</span> |
| <span class="w"> </span><span class="kd">private</span><span class="w"> </span><span class="kd">final</span><span class="w"> </span><span class="n">ConcurrentMap</span><span class="o"><</span><span class="n">FlightDescriptor</span><span class="p">,</span><span class="w"> </span><span class="n">Dataset</span><span class="o">></span><span class="w"> </span><span class="n">datasets</span><span class="p">;</span> |
| <span class="w"> </span><span class="kd">public</span><span class="w"> </span><span class="nf">CookbookProducer</span><span class="p">(</span><span class="n">BufferAllocator</span><span class="w"> </span><span class="n">allocator</span><span class="p">,</span><span class="w"> </span><span class="n">Location</span><span class="w"> </span><span class="n">location</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="k">this</span><span class="p">.</span><span class="na">allocator</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">allocator</span><span class="p">;</span> |
| <span class="w"> </span><span class="k">this</span><span class="p">.</span><span class="na">location</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">location</span><span class="p">;</span> |
| <span class="w"> </span><span class="k">this</span><span class="p">.</span><span class="na">datasets</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">ConcurrentHashMap</span><span class="o"><></span><span class="p">();</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="nd">@Override</span> |
| <span class="w"> </span><span class="kd">public</span><span class="w"> </span><span class="n">Runnable</span><span class="w"> </span><span class="nf">acceptPut</span><span class="p">(</span><span class="n">CallContext</span><span class="w"> </span><span class="n">context</span><span class="p">,</span><span class="w"> </span><span class="n">FlightStream</span><span class="w"> </span><span class="n">flightStream</span><span class="p">,</span><span class="w"> </span><span class="n">StreamListener</span><span class="o"><</span><span class="n">PutResult</span><span class="o">></span><span class="w"> </span><span class="n">ackStream</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">List</span><span class="o"><</span><span class="n">ArrowRecordBatch</span><span class="o">></span><span class="w"> </span><span class="n">batches</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">ArrayList</span><span class="o"><></span><span class="p">();</span> |
| <span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="p">()</span><span class="w"> </span><span class="o">-></span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="kt">long</span><span class="w"> </span><span class="n">rows</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="mi">0</span><span class="p">;</span> |
| <span class="w"> </span><span class="n">VectorUnloader</span><span class="w"> </span><span class="n">unloader</span><span class="p">;</span> |
| <span class="w"> </span><span class="k">while</span><span class="w"> </span><span class="p">(</span><span class="n">flightStream</span><span class="p">.</span><span class="na">next</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">unloader</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">VectorUnloader</span><span class="p">(</span><span class="n">flightStream</span><span class="p">.</span><span class="na">getRoot</span><span class="p">());</span> |
| <span class="w"> </span><span class="kd">final</span><span class="w"> </span><span class="n">ArrowRecordBatch</span><span class="w"> </span><span class="n">arb</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">unloader</span><span class="p">.</span><span class="na">getRecordBatch</span><span class="p">();</span> |
| <span class="w"> </span><span class="n">batches</span><span class="p">.</span><span class="na">add</span><span class="p">(</span><span class="n">arb</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">rows</span><span class="w"> </span><span class="o">+=</span><span class="w"> </span><span class="n">flightStream</span><span class="p">.</span><span class="na">getRoot</span><span class="p">().</span><span class="na">getRowCount</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="n">Dataset</span><span class="w"> </span><span class="n">dataset</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">Dataset</span><span class="p">(</span><span class="n">batches</span><span class="p">,</span><span class="w"> </span><span class="n">flightStream</span><span class="p">.</span><span class="na">getSchema</span><span class="p">(),</span><span class="w"> </span><span class="n">rows</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">datasets</span><span class="p">.</span><span class="na">put</span><span class="p">(</span><span class="n">flightStream</span><span class="p">.</span><span class="na">getDescriptor</span><span class="p">(),</span><span class="w"> </span><span class="n">dataset</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">ackStream</span><span class="p">.</span><span class="na">onCompleted</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">};</span> |
| <span class="w"> </span><span class="p">}</span> |
| |
| <span class="w"> </span><span class="nd">@Override</span> |
| <span class="w"> </span><span class="kd">public</span><span class="w"> </span><span class="kt">void</span><span class="w"> </span><span class="nf">getStream</span><span class="p">(</span><span class="n">CallContext</span><span class="w"> </span><span class="n">context</span><span class="p">,</span><span class="w"> </span><span class="n">Ticket</span><span class="w"> </span><span class="n">ticket</span><span class="p">,</span><span class="w"> </span><span class="n">ServerStreamListener</span><span class="w"> </span><span class="n">listener</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">FlightDescriptor</span><span class="w"> </span><span class="n">flightDescriptor</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">FlightDescriptor</span><span class="p">.</span><span class="na">path</span><span class="p">(</span> |
| <span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">String</span><span class="p">(</span><span class="n">ticket</span><span class="p">.</span><span class="na">getBytes</span><span class="p">(),</span><span class="w"> </span><span class="n">StandardCharsets</span><span class="p">.</span><span class="na">UTF_8</span><span class="p">));</span> |
| <span class="w"> </span><span class="n">Dataset</span><span class="w"> </span><span class="n">dataset</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">this</span><span class="p">.</span><span class="na">datasets</span><span class="p">.</span><span class="na">get</span><span class="p">(</span><span class="n">flightDescriptor</span><span class="p">);</span> |
| <span class="w"> </span><span class="k">if</span><span class="w"> </span><span class="p">(</span><span class="n">dataset</span><span class="w"> </span><span class="o">==</span><span class="w"> </span><span class="kc">null</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="k">throw</span><span class="w"> </span><span class="n">CallStatus</span><span class="p">.</span><span class="na">NOT_FOUND</span><span class="p">.</span><span class="na">withDescription</span><span class="p">(</span><span class="s">"Unknown descriptor"</span><span class="p">).</span><span class="na">toRuntimeException</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="k">try</span><span class="w"> </span><span class="p">(</span><span class="n">VectorSchemaRoot</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">VectorSchemaRoot</span><span class="p">.</span><span class="na">create</span><span class="p">(</span> |
| <span class="w"> </span><span class="k">this</span><span class="p">.</span><span class="na">datasets</span><span class="p">.</span><span class="na">get</span><span class="p">(</span><span class="n">flightDescriptor</span><span class="p">).</span><span class="na">getSchema</span><span class="p">(),</span><span class="w"> </span><span class="n">allocator</span><span class="p">))</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">VectorLoader</span><span class="w"> </span><span class="n">loader</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">VectorLoader</span><span class="p">(</span><span class="n">root</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">start</span><span class="p">(</span><span class="n">root</span><span class="p">);</span> |
| <span class="w"> </span><span class="k">for</span><span class="w"> </span><span class="p">(</span><span class="n">ArrowRecordBatch</span><span class="w"> </span><span class="n">arrowRecordBatch</span><span class="w"> </span><span class="p">:</span><span class="w"> </span><span class="k">this</span><span class="p">.</span><span class="na">datasets</span><span class="p">.</span><span class="na">get</span><span class="p">(</span><span class="n">flightDescriptor</span><span class="p">).</span><span class="na">getBatches</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">loader</span><span class="p">.</span><span class="na">load</span><span class="p">(</span><span class="n">arrowRecordBatch</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">putNext</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">completed</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="p">}</span> |
| |
| <span class="w"> </span><span class="nd">@Override</span> |
| <span class="w"> </span><span class="kd">public</span><span class="w"> </span><span class="kt">void</span><span class="w"> </span><span class="nf">doAction</span><span class="p">(</span><span class="n">CallContext</span><span class="w"> </span><span class="n">context</span><span class="p">,</span><span class="w"> </span><span class="n">Action</span><span class="w"> </span><span class="n">action</span><span class="p">,</span><span class="w"> </span><span class="n">StreamListener</span><span class="o"><</span><span class="n">Result</span><span class="o">></span><span class="w"> </span><span class="n">listener</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">FlightDescriptor</span><span class="w"> </span><span class="n">flightDescriptor</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">FlightDescriptor</span><span class="p">.</span><span class="na">path</span><span class="p">(</span> |
| <span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">String</span><span class="p">(</span><span class="n">action</span><span class="p">.</span><span class="na">getBody</span><span class="p">(),</span><span class="w"> </span><span class="n">StandardCharsets</span><span class="p">.</span><span class="na">UTF_8</span><span class="p">));</span> |
| <span class="w"> </span><span class="k">switch</span><span class="w"> </span><span class="p">(</span><span class="n">action</span><span class="p">.</span><span class="na">getType</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="k">case</span><span class="w"> </span><span class="s">"DELETE"</span><span class="p">:</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">Dataset</span><span class="w"> </span><span class="n">removed</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">datasets</span><span class="p">.</span><span class="na">remove</span><span class="p">(</span><span class="n">flightDescriptor</span><span class="p">);</span> |
| <span class="w"> </span><span class="k">if</span><span class="w"> </span><span class="p">(</span><span class="n">removed</span><span class="w"> </span><span class="o">!=</span><span class="w"> </span><span class="kc">null</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="k">try</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">removed</span><span class="p">.</span><span class="na">close</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">}</span><span class="w"> </span><span class="k">catch</span><span class="w"> </span><span class="p">(</span><span class="n">Exception</span><span class="w"> </span><span class="n">e</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">onError</span><span class="p">(</span><span class="n">CallStatus</span><span class="p">.</span><span class="na">INTERNAL</span> |
| <span class="w"> </span><span class="p">.</span><span class="na">withDescription</span><span class="p">(</span><span class="n">e</span><span class="p">.</span><span class="na">toString</span><span class="p">())</span> |
| <span class="w"> </span><span class="p">.</span><span class="na">toRuntimeException</span><span class="p">());</span> |
| <span class="w"> </span><span class="k">return</span><span class="p">;</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="n">Result</span><span class="w"> </span><span class="n">result</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">Result</span><span class="p">(</span><span class="s">"Delete completed"</span><span class="p">.</span><span class="na">getBytes</span><span class="p">(</span><span class="n">StandardCharsets</span><span class="p">.</span><span class="na">UTF_8</span><span class="p">));</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">onNext</span><span class="p">(</span><span class="n">result</span><span class="p">);</span> |
| <span class="w"> </span><span class="p">}</span><span class="w"> </span><span class="k">else</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">Result</span><span class="w"> </span><span class="n">result</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">Result</span><span class="p">(</span><span class="s">"Delete not completed. Reason: Key did not exist."</span> |
| <span class="w"> </span><span class="p">.</span><span class="na">getBytes</span><span class="p">(</span><span class="n">StandardCharsets</span><span class="p">.</span><span class="na">UTF_8</span><span class="p">));</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">onNext</span><span class="p">(</span><span class="n">result</span><span class="p">);</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">onCompleted</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="p">}</span> |
| |
| <span class="w"> </span><span class="nd">@Override</span> |
| <span class="w"> </span><span class="kd">public</span><span class="w"> </span><span class="n">FlightInfo</span><span class="w"> </span><span class="nf">getFlightInfo</span><span class="p">(</span><span class="n">CallContext</span><span class="w"> </span><span class="n">context</span><span class="p">,</span><span class="w"> </span><span class="n">FlightDescriptor</span><span class="w"> </span><span class="n">descriptor</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">FlightEndpoint</span><span class="w"> </span><span class="n">flightEndpoint</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">FlightEndpoint</span><span class="p">(</span> |
| <span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">Ticket</span><span class="p">(</span><span class="n">descriptor</span><span class="p">.</span><span class="na">getPath</span><span class="p">().</span><span class="na">get</span><span class="p">(</span><span class="mi">0</span><span class="p">).</span><span class="na">getBytes</span><span class="p">(</span><span class="n">StandardCharsets</span><span class="p">.</span><span class="na">UTF_8</span><span class="p">)),</span><span class="w"> </span><span class="n">location</span><span class="p">);</span> |
| <span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">FlightInfo</span><span class="p">(</span> |
| <span class="w"> </span><span class="n">datasets</span><span class="p">.</span><span class="na">get</span><span class="p">(</span><span class="n">descriptor</span><span class="p">).</span><span class="na">getSchema</span><span class="p">(),</span> |
| <span class="w"> </span><span class="n">descriptor</span><span class="p">,</span> |
| <span class="w"> </span><span class="n">Collections</span><span class="p">.</span><span class="na">singletonList</span><span class="p">(</span><span class="n">flightEndpoint</span><span class="p">),</span> |
| <span class="w"> </span><span class="cm">/*bytes=*/</span><span class="o">-</span><span class="mi">1</span><span class="p">,</span> |
| <span class="w"> </span><span class="n">datasets</span><span class="p">.</span><span class="na">get</span><span class="p">(</span><span class="n">descriptor</span><span class="p">).</span><span class="na">getRows</span><span class="p">()</span> |
| <span class="w"> </span><span class="p">);</span> |
| <span class="w"> </span><span class="p">}</span> |
| |
| <span class="w"> </span><span class="nd">@Override</span> |
| <span class="w"> </span><span class="kd">public</span><span class="w"> </span><span class="kt">void</span><span class="w"> </span><span class="nf">listFlights</span><span class="p">(</span><span class="n">CallContext</span><span class="w"> </span><span class="n">context</span><span class="p">,</span><span class="w"> </span><span class="n">Criteria</span><span class="w"> </span><span class="n">criteria</span><span class="p">,</span><span class="w"> </span><span class="n">StreamListener</span><span class="o"><</span><span class="n">FlightInfo</span><span class="o">></span><span class="w"> </span><span class="n">listener</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">datasets</span><span class="p">.</span><span class="na">forEach</span><span class="p">((</span><span class="n">k</span><span class="p">,</span><span class="w"> </span><span class="n">v</span><span class="p">)</span><span class="w"> </span><span class="o">-></span><span class="w"> </span><span class="p">{</span><span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">onNext</span><span class="p">(</span><span class="n">getFlightInfo</span><span class="p">(</span><span class="kc">null</span><span class="p">,</span><span class="w"> </span><span class="n">k</span><span class="p">));</span><span class="w"> </span><span class="p">});</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">onCompleted</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">}</span> |
| |
| <span class="w"> </span><span class="nd">@Override</span> |
| <span class="w"> </span><span class="kd">public</span><span class="w"> </span><span class="kt">void</span><span class="w"> </span><span class="nf">close</span><span class="p">()</span><span class="w"> </span><span class="kd">throws</span><span class="w"> </span><span class="n">Exception</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">AutoCloseables</span><span class="p">.</span><span class="na">close</span><span class="p">(</span><span class="n">datasets</span><span class="p">.</span><span class="na">values</span><span class="p">());</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="p">}</span> |
| <span class="n">Location</span><span class="w"> </span><span class="n">location</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">Location</span><span class="p">.</span><span class="na">forGrpcInsecure</span><span class="p">(</span><span class="s">"0.0.0.0"</span><span class="p">,</span><span class="w"> </span><span class="mi">33333</span><span class="p">);</span> |
| <span class="k">try</span><span class="w"> </span><span class="p">(</span><span class="n">BufferAllocator</span><span class="w"> </span><span class="n">allocator</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">RootAllocator</span><span class="p">()){</span> |
| <span class="w"> </span><span class="c1">// Server</span> |
| <span class="w"> </span><span class="k">try</span><span class="p">(</span><span class="kd">final</span><span class="w"> </span><span class="n">CookbookProducer</span><span class="w"> </span><span class="n">producer</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">CookbookProducer</span><span class="p">(</span><span class="n">allocator</span><span class="p">,</span><span class="w"> </span><span class="n">location</span><span class="p">);</span> |
| <span class="w"> </span><span class="kd">final</span><span class="w"> </span><span class="n">FlightServer</span><span class="w"> </span><span class="n">flightServer</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">FlightServer</span><span class="p">.</span><span class="na">builder</span><span class="p">(</span><span class="n">allocator</span><span class="p">,</span><span class="w"> </span><span class="n">location</span><span class="p">,</span><span class="w"> </span><span class="n">producer</span><span class="p">).</span><span class="na">build</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="k">try</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">flightServer</span><span class="p">.</span><span class="na">start</span><span class="p">();</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"S1: Server (Location): Listening on port "</span><span class="w"> </span><span class="o">+</span><span class="w"> </span><span class="n">flightServer</span><span class="p">.</span><span class="na">getPort</span><span class="p">());</span> |
| <span class="w"> </span><span class="p">}</span><span class="w"> </span><span class="k">catch</span><span class="w"> </span><span class="p">(</span><span class="n">IOException</span><span class="w"> </span><span class="n">e</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="k">throw</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">RuntimeException</span><span class="p">(</span><span class="n">e</span><span class="p">);</span> |
| <span class="w"> </span><span class="p">}</span> |
| |
| <span class="w"> </span><span class="c1">// Client</span> |
| <span class="w"> </span><span class="k">try</span><span class="w"> </span><span class="p">(</span><span class="n">FlightClient</span><span class="w"> </span><span class="n">flightClient</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">FlightClient</span><span class="p">.</span><span class="na">builder</span><span class="p">(</span><span class="n">allocator</span><span class="p">,</span><span class="w"> </span><span class="n">location</span><span class="p">).</span><span class="na">build</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"C1: Client (Location): Connected to "</span><span class="w"> </span><span class="o">+</span><span class="w"> </span><span class="n">location</span><span class="p">.</span><span class="na">getUri</span><span class="p">());</span> |
| |
| <span class="w"> </span><span class="c1">// Populate data</span> |
| <span class="w"> </span><span class="n">Schema</span><span class="w"> </span><span class="n">schema</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">Schema</span><span class="p">(</span><span class="n">Arrays</span><span class="p">.</span><span class="na">asList</span><span class="p">(</span> |
| <span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">Field</span><span class="p">(</span><span class="s">"name"</span><span class="p">,</span><span class="w"> </span><span class="n">FieldType</span><span class="p">.</span><span class="na">nullable</span><span class="p">(</span><span class="k">new</span><span class="w"> </span><span class="n">ArrowType</span><span class="p">.</span><span class="na">Utf8</span><span class="p">()),</span><span class="w"> </span><span class="kc">null</span><span class="p">)));</span> |
| <span class="w"> </span><span class="k">try</span><span class="p">(</span><span class="n">VectorSchemaRoot</span><span class="w"> </span><span class="n">vectorSchemaRoot</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">VectorSchemaRoot</span><span class="p">.</span><span class="na">create</span><span class="p">(</span><span class="n">schema</span><span class="p">,</span><span class="w"> </span><span class="n">allocator</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">VarCharVector</span><span class="w"> </span><span class="n">varCharVector</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="p">(</span><span class="n">VarCharVector</span><span class="p">)</span><span class="w"> </span><span class="n">vectorSchemaRoot</span><span class="p">.</span><span class="na">getVector</span><span class="p">(</span><span class="s">"name"</span><span class="p">))</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">varCharVector</span><span class="p">.</span><span class="na">allocateNew</span><span class="p">(</span><span class="mi">3</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">varCharVector</span><span class="p">.</span><span class="na">set</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span><span class="w"> </span><span class="s">"Ronald"</span><span class="p">.</span><span class="na">getBytes</span><span class="p">());</span> |
| <span class="w"> </span><span class="n">varCharVector</span><span class="p">.</span><span class="na">set</span><span class="p">(</span><span class="mi">1</span><span class="p">,</span><span class="w"> </span><span class="s">"David"</span><span class="p">.</span><span class="na">getBytes</span><span class="p">());</span> |
| <span class="w"> </span><span class="n">varCharVector</span><span class="p">.</span><span class="na">set</span><span class="p">(</span><span class="mi">2</span><span class="p">,</span><span class="w"> </span><span class="s">"Francisco"</span><span class="p">.</span><span class="na">getBytes</span><span class="p">());</span> |
| <span class="w"> </span><span class="n">vectorSchemaRoot</span><span class="p">.</span><span class="na">setRowCount</span><span class="p">(</span><span class="mi">3</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">FlightClient</span><span class="p">.</span><span class="na">ClientStreamListener</span><span class="w"> </span><span class="n">listener</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">flightClient</span><span class="p">.</span><span class="na">startPut</span><span class="p">(</span> |
| <span class="w"> </span><span class="n">FlightDescriptor</span><span class="p">.</span><span class="na">path</span><span class="p">(</span><span class="s">"profiles"</span><span class="p">),</span> |
| <span class="w"> </span><span class="n">vectorSchemaRoot</span><span class="p">,</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">AsyncPutListener</span><span class="p">());</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">putNext</span><span class="p">();</span> |
| <span class="w"> </span><span class="n">varCharVector</span><span class="p">.</span><span class="na">set</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span><span class="w"> </span><span class="s">"Manuel"</span><span class="p">.</span><span class="na">getBytes</span><span class="p">());</span> |
| <span class="w"> </span><span class="n">varCharVector</span><span class="p">.</span><span class="na">set</span><span class="p">(</span><span class="mi">1</span><span class="p">,</span><span class="w"> </span><span class="s">"Felipe"</span><span class="p">.</span><span class="na">getBytes</span><span class="p">());</span> |
| <span class="w"> </span><span class="n">varCharVector</span><span class="p">.</span><span class="na">set</span><span class="p">(</span><span class="mi">2</span><span class="p">,</span><span class="w"> </span><span class="s">"JJ"</span><span class="p">.</span><span class="na">getBytes</span><span class="p">());</span> |
| <span class="w"> </span><span class="n">vectorSchemaRoot</span><span class="p">.</span><span class="na">setRowCount</span><span class="p">(</span><span class="mi">3</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">putNext</span><span class="p">();</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">completed</span><span class="p">();</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">getResult</span><span class="p">();</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"C2: Client (Populate Data): Wrote 2 batches with 3 rows each"</span><span class="p">);</span> |
| <span class="w"> </span><span class="p">}</span> |
| |
| <span class="w"> </span><span class="c1">// Get metadata information</span> |
| <span class="w"> </span><span class="n">FlightInfo</span><span class="w"> </span><span class="n">flightInfo</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">flightClient</span><span class="p">.</span><span class="na">getInfo</span><span class="p">(</span><span class="n">FlightDescriptor</span><span class="p">.</span><span class="na">path</span><span class="p">(</span><span class="s">"profiles"</span><span class="p">));</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"C3: Client (Get Metadata): "</span><span class="w"> </span><span class="o">+</span><span class="w"> </span><span class="n">flightInfo</span><span class="p">);</span> |
| |
| <span class="w"> </span><span class="c1">// Get data information</span> |
| <span class="w"> </span><span class="k">try</span><span class="p">(</span><span class="n">FlightStream</span><span class="w"> </span><span class="n">flightStream</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">flightClient</span><span class="p">.</span><span class="na">getStream</span><span class="p">(</span><span class="n">flightInfo</span><span class="p">.</span><span class="na">getEndpoints</span><span class="p">().</span><span class="na">get</span><span class="p">(</span><span class="mi">0</span><span class="p">).</span><span class="na">getTicket</span><span class="p">()))</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="kt">int</span><span class="w"> </span><span class="n">batch</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="mi">0</span><span class="p">;</span> |
| <span class="w"> </span><span class="k">try</span><span class="w"> </span><span class="p">(</span><span class="n">VectorSchemaRoot</span><span class="w"> </span><span class="n">vectorSchemaRootReceived</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">flightStream</span><span class="p">.</span><span class="na">getRoot</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"C4: Client (Get Stream):"</span><span class="p">);</span> |
| <span class="w"> </span><span class="k">while</span><span class="w"> </span><span class="p">(</span><span class="n">flightStream</span><span class="p">.</span><span class="na">next</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">batch</span><span class="o">++</span><span class="p">;</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"Client Received batch #"</span><span class="w"> </span><span class="o">+</span><span class="w"> </span><span class="n">batch</span><span class="w"> </span><span class="o">+</span><span class="w"> </span><span class="s">", Data:"</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">print</span><span class="p">(</span><span class="n">vectorSchemaRootReceived</span><span class="p">.</span><span class="na">contentToTSVString</span><span class="p">());</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="p">}</span><span class="w"> </span><span class="k">catch</span><span class="w"> </span><span class="p">(</span><span class="n">Exception</span><span class="w"> </span><span class="n">e</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">e</span><span class="p">.</span><span class="na">printStackTrace</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">}</span> |
| |
| <span class="w"> </span><span class="c1">// Get all metadata information</span> |
| <span class="w"> </span><span class="n">Iterable</span><span class="o"><</span><span class="n">FlightInfo</span><span class="o">></span><span class="w"> </span><span class="n">flightInfosBefore</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">flightClient</span><span class="p">.</span><span class="na">listFlights</span><span class="p">(</span><span class="n">Criteria</span><span class="p">.</span><span class="na">ALL</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">print</span><span class="p">(</span><span class="s">"C5: Client (List Flights Info): "</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">flightInfosBefore</span><span class="p">.</span><span class="na">forEach</span><span class="p">(</span><span class="n">t</span><span class="w"> </span><span class="o">-></span><span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="n">t</span><span class="p">));</span> |
| |
| <span class="w"> </span><span class="c1">// Do delete action</span> |
| <span class="w"> </span><span class="n">Iterator</span><span class="o"><</span><span class="n">Result</span><span class="o">></span><span class="w"> </span><span class="n">deleteActionResult</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">flightClient</span><span class="p">.</span><span class="na">doAction</span><span class="p">(</span><span class="k">new</span><span class="w"> </span><span class="n">Action</span><span class="p">(</span><span class="s">"DELETE"</span><span class="p">,</span> |
| <span class="w"> </span><span class="n">FlightDescriptor</span><span class="p">.</span><span class="na">path</span><span class="p">(</span><span class="s">"profiles"</span><span class="p">).</span><span class="na">getPath</span><span class="p">().</span><span class="na">get</span><span class="p">(</span><span class="mi">0</span><span class="p">).</span><span class="na">getBytes</span><span class="p">(</span><span class="n">StandardCharsets</span><span class="p">.</span><span class="na">UTF_8</span><span class="p">)));</span> |
| <span class="w"> </span><span class="k">while</span><span class="w"> </span><span class="p">(</span><span class="n">deleteActionResult</span><span class="p">.</span><span class="na">hasNext</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">Result</span><span class="w"> </span><span class="n">result</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">deleteActionResult</span><span class="p">.</span><span class="na">next</span><span class="p">();</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"C6: Client (Do Delete Action): "</span><span class="w"> </span><span class="o">+</span> |
| <span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">String</span><span class="p">(</span><span class="n">result</span><span class="p">.</span><span class="na">getBody</span><span class="p">(),</span><span class="w"> </span><span class="n">StandardCharsets</span><span class="p">.</span><span class="na">UTF_8</span><span class="p">));</span> |
| <span class="w"> </span><span class="p">}</span> |
| |
| <span class="w"> </span><span class="c1">// Get all metadata information (to validate detele action)</span> |
| <span class="w"> </span><span class="n">Iterable</span><span class="o"><</span><span class="n">FlightInfo</span><span class="o">></span><span class="w"> </span><span class="n">flightInfos</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">flightClient</span><span class="p">.</span><span class="na">listFlights</span><span class="p">(</span><span class="n">Criteria</span><span class="p">.</span><span class="na">ALL</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">flightInfos</span><span class="p">.</span><span class="na">forEach</span><span class="p">(</span><span class="n">t</span><span class="w"> </span><span class="o">-></span><span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="n">t</span><span class="p">));</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"C7: Client (List Flights Info): After delete - No records"</span><span class="p">);</span> |
| |
| <span class="w"> </span><span class="c1">// Server shut down</span> |
| <span class="w"> </span><span class="n">flightServer</span><span class="p">.</span><span class="na">shutdown</span><span class="p">();</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"C8: Server shut down successfully"</span><span class="p">);</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="p">}</span><span class="w"> </span><span class="k">catch</span><span class="w"> </span><span class="p">(</span><span class="n">Exception</span><span class="w"> </span><span class="n">e</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">e</span><span class="p">.</span><span class="na">printStackTrace</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="p">}</span> |
| </pre></div> |
| </div> |
| <div class="highlight-none notranslate"><div class="highlight"><pre><span></span>S1: Server (Location): Listening on port 33333 |
| C1: Client (Location): Connected to grpc+tcp://0.0.0.0:33333 |
| C2: Client (Populate Data): Wrote 2 batches with 3 rows each |
| C3: Client (Get Metadata): FlightInfo{schema=Schema<name: Utf8>, descriptor=profiles, endpoints=[FlightEndpoint{locations=[Location{uri=grpc+tcp://0.0.0.0:33333}], ticket=org.apache.arrow.flight.Ticket@58871b0a, expirationTime=(none), appMetadata=(none)}], bytes=-1, records=6, ordered=false, appMetadata=(none)} |
| C4: Client (Get Stream): |
| Client Received batch #1, Data: |
| name |
| Ronald |
| David |
| Francisco |
| Client Received batch #2, Data: |
| name |
| Manuel |
| Felipe |
| JJ |
| C5: Client (List Flights Info): FlightInfo{schema=Schema<name: Utf8>, descriptor=profiles, endpoints=[FlightEndpoint{locations=[Location{uri=grpc+tcp://0.0.0.0:33333}], ticket=org.apache.arrow.flight.Ticket@58871b0a, expirationTime=(none), appMetadata=(none)}], bytes=-1, records=6, ordered=false, appMetadata=(none)} |
| C6: Client (Do Delete Action): Delete completed |
| C7: Client (List Flights Info): After delete - No records |
| C8: Server shut down successfully |
| </pre></div> |
| </div> |
| <p>Let explain our code in more detail.</p> |
| </section> |
| <section id="start-flight-server"> |
| <h3><a class="toc-backref" href="#id4" role="doc-backlink">Start Flight Server</a><a class="headerlink" href="#start-flight-server" title="Link to this heading">¶</a></h3> |
| <p>First, we’ll start our server:</p> |
| <div class="highlight-java notranslate"><div class="highlight"><pre><span></span><span class="k">try</span><span class="p">(</span><span class="n">FlightServer</span><span class="w"> </span><span class="n">flightServer</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">FlightServer</span><span class="p">.</span><span class="na">builder</span><span class="p">(</span><span class="n">allocator</span><span class="p">,</span><span class="w"> </span><span class="n">location</span><span class="p">,</span> |
| <span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">CookbookProducer</span><span class="p">(</span><span class="n">allocator</span><span class="p">,</span><span class="w"> </span><span class="n">location</span><span class="p">)).</span><span class="na">build</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="k">try</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">flightServer</span><span class="p">.</span><span class="na">start</span><span class="p">();</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"S1: Server (Location): Listening on port "</span><span class="w"> </span><span class="o">+</span><span class="w"> </span><span class="n">flightServer</span><span class="p">.</span><span class="na">getPort</span><span class="p">());</span> |
| <span class="w"> </span><span class="p">}</span><span class="w"> </span><span class="k">catch</span><span class="w"> </span><span class="p">(</span><span class="n">IOException</span><span class="w"> </span><span class="n">e</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">e</span><span class="p">.</span><span class="na">printStackTrace</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">}</span> |
| </pre></div> |
| </div> |
| <div class="highlight-shell notranslate"><div class="highlight"><pre><span></span>S1:<span class="w"> </span>Server<span class="w"> </span><span class="o">(</span>Location<span class="o">)</span>:<span class="w"> </span>Listening<span class="w"> </span>on<span class="w"> </span>port<span class="w"> </span><span class="m">33333</span> |
| </pre></div> |
| </div> |
| </section> |
| <section id="connect-to-flight-server"> |
| <h3><a class="toc-backref" href="#id5" role="doc-backlink">Connect to Flight Server</a><a class="headerlink" href="#connect-to-flight-server" title="Link to this heading">¶</a></h3> |
| <p>We can then create a client and connect to the server:</p> |
| <div class="highlight-java notranslate"><div class="highlight"><pre><span></span><span class="k">try</span><span class="w"> </span><span class="p">(</span><span class="n">FlightClient</span><span class="w"> </span><span class="n">flightClient</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">FlightClient</span><span class="p">.</span><span class="na">builder</span><span class="p">(</span><span class="n">allocator</span><span class="p">,</span><span class="w"> </span><span class="n">location</span><span class="p">).</span><span class="na">build</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"C1: Client (Location): Connected to "</span><span class="w"> </span><span class="o">+</span><span class="w"> </span><span class="n">location</span><span class="p">.</span><span class="na">getUri</span><span class="p">());</span> |
| </pre></div> |
| </div> |
| <div class="highlight-shell notranslate"><div class="highlight"><pre><span></span>C1:<span class="w"> </span>Client<span class="w"> </span><span class="o">(</span>Location<span class="o">)</span>:<span class="w"> </span>Connected<span class="w"> </span>to<span class="w"> </span>grpc+tcp://0.0.0.0:33333 |
| </pre></div> |
| </div> |
| </section> |
| <section id="put-data"> |
| <h3><a class="toc-backref" href="#id6" role="doc-backlink">Put Data</a><a class="headerlink" href="#put-data" title="Link to this heading">¶</a></h3> |
| <p>First, we’ll create and upload a vector schema root, which will get stored in a |
| memory by the server.</p> |
| <div class="highlight-java notranslate"><div class="highlight"><pre><span></span><span class="c1">// Server</span> |
| <span class="kd">public</span><span class="w"> </span><span class="n">Runnable</span><span class="w"> </span><span class="nf">acceptPut</span><span class="p">(</span><span class="n">CallContext</span><span class="w"> </span><span class="n">context</span><span class="p">,</span><span class="w"> </span><span class="n">FlightStream</span><span class="w"> </span><span class="n">flightStream</span><span class="p">,</span><span class="w"> </span><span class="n">StreamListener</span><span class="o"><</span><span class="n">PutResult</span><span class="o">></span><span class="w"> </span><span class="n">ackStream</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">List</span><span class="o"><</span><span class="n">ArrowRecordBatch</span><span class="o">></span><span class="w"> </span><span class="n">batches</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">ArrayList</span><span class="o"><></span><span class="p">();</span> |
| <span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="p">()</span><span class="w"> </span><span class="o">-></span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="kt">long</span><span class="w"> </span><span class="n">rows</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="mi">0</span><span class="p">;</span> |
| <span class="w"> </span><span class="n">VectorUnloader</span><span class="w"> </span><span class="n">unloader</span><span class="p">;</span> |
| <span class="w"> </span><span class="k">while</span><span class="w"> </span><span class="p">(</span><span class="n">flightStream</span><span class="p">.</span><span class="na">next</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">unloader</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">VectorUnloader</span><span class="p">(</span><span class="n">flightStream</span><span class="p">.</span><span class="na">getRoot</span><span class="p">());</span> |
| <span class="w"> </span><span class="k">try</span><span class="w"> </span><span class="p">(</span><span class="kd">final</span><span class="w"> </span><span class="n">ArrowRecordBatch</span><span class="w"> </span><span class="n">arb</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">unloader</span><span class="p">.</span><span class="na">getRecordBatch</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">batches</span><span class="p">.</span><span class="na">add</span><span class="p">(</span><span class="n">arb</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">rows</span><span class="w"> </span><span class="o">+=</span><span class="w"> </span><span class="n">flightStream</span><span class="p">.</span><span class="na">getRoot</span><span class="p">().</span><span class="na">getRowCount</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="n">Dataset</span><span class="w"> </span><span class="n">dataset</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">Dataset</span><span class="p">(</span><span class="n">batches</span><span class="p">,</span><span class="w"> </span><span class="n">flightStream</span><span class="p">.</span><span class="na">getSchema</span><span class="p">(),</span><span class="w"> </span><span class="n">rows</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">datasets</span><span class="p">.</span><span class="na">put</span><span class="p">(</span><span class="n">flightStream</span><span class="p">.</span><span class="na">getDescriptor</span><span class="p">(),</span><span class="w"> </span><span class="n">dataset</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">ackStream</span><span class="p">.</span><span class="na">onCompleted</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">};</span> |
| <span class="p">}</span> |
| |
| <span class="c1">// Client</span> |
| <span class="n">Schema</span><span class="w"> </span><span class="n">schema</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">Schema</span><span class="p">(</span><span class="n">Arrays</span><span class="p">.</span><span class="na">asList</span><span class="p">(</span> |
| <span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">Field</span><span class="p">(</span><span class="s">"name"</span><span class="p">,</span><span class="w"> </span><span class="n">FieldType</span><span class="p">.</span><span class="na">nullable</span><span class="p">(</span><span class="k">new</span><span class="w"> </span><span class="n">ArrowType</span><span class="p">.</span><span class="na">Utf8</span><span class="p">()),</span><span class="w"> </span><span class="kc">null</span><span class="p">)));</span> |
| <span class="k">try</span><span class="p">(</span><span class="n">VectorSchemaRoot</span><span class="w"> </span><span class="n">vectorSchemaRoot</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">VectorSchemaRoot</span><span class="p">.</span><span class="na">create</span><span class="p">(</span><span class="n">schema</span><span class="p">,</span><span class="w"> </span><span class="n">allocator</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">VarCharVector</span><span class="w"> </span><span class="n">varCharVector</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="p">(</span><span class="n">VarCharVector</span><span class="p">)</span><span class="w"> </span><span class="n">vectorSchemaRoot</span><span class="p">.</span><span class="na">getVector</span><span class="p">(</span><span class="s">"name"</span><span class="p">))</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">varCharVector</span><span class="p">.</span><span class="na">allocateNew</span><span class="p">(</span><span class="mi">3</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">varCharVector</span><span class="p">.</span><span class="na">set</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span><span class="w"> </span><span class="s">"Ronald"</span><span class="p">.</span><span class="na">getBytes</span><span class="p">());</span> |
| <span class="w"> </span><span class="n">varCharVector</span><span class="p">.</span><span class="na">set</span><span class="p">(</span><span class="mi">1</span><span class="p">,</span><span class="w"> </span><span class="s">"David"</span><span class="p">.</span><span class="na">getBytes</span><span class="p">());</span> |
| <span class="w"> </span><span class="n">varCharVector</span><span class="p">.</span><span class="na">set</span><span class="p">(</span><span class="mi">2</span><span class="p">,</span><span class="w"> </span><span class="s">"Francisco"</span><span class="p">.</span><span class="na">getBytes</span><span class="p">());</span> |
| <span class="w"> </span><span class="n">vectorSchemaRoot</span><span class="p">.</span><span class="na">setRowCount</span><span class="p">(</span><span class="mi">3</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">FlightClient</span><span class="p">.</span><span class="na">ClientStreamListener</span><span class="w"> </span><span class="n">listener</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">flightClient</span><span class="p">.</span><span class="na">startPut</span><span class="p">(</span> |
| <span class="w"> </span><span class="n">FlightDescriptor</span><span class="p">.</span><span class="na">path</span><span class="p">(</span><span class="s">"profiles"</span><span class="p">),</span> |
| <span class="w"> </span><span class="n">vectorSchemaRoot</span><span class="p">,</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">AsyncPutListener</span><span class="p">());</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">putNext</span><span class="p">();</span> |
| <span class="w"> </span><span class="n">varCharVector</span><span class="p">.</span><span class="na">set</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span><span class="w"> </span><span class="s">"Manuel"</span><span class="p">.</span><span class="na">getBytes</span><span class="p">());</span> |
| <span class="w"> </span><span class="n">varCharVector</span><span class="p">.</span><span class="na">set</span><span class="p">(</span><span class="mi">1</span><span class="p">,</span><span class="w"> </span><span class="s">"Felipe"</span><span class="p">.</span><span class="na">getBytes</span><span class="p">());</span> |
| <span class="w"> </span><span class="n">varCharVector</span><span class="p">.</span><span class="na">set</span><span class="p">(</span><span class="mi">2</span><span class="p">,</span><span class="w"> </span><span class="s">"JJ"</span><span class="p">.</span><span class="na">getBytes</span><span class="p">());</span> |
| <span class="w"> </span><span class="n">vectorSchemaRoot</span><span class="p">.</span><span class="na">setRowCount</span><span class="p">(</span><span class="mi">3</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">putNext</span><span class="p">();</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">completed</span><span class="p">();</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">getResult</span><span class="p">();</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"C2: Client (Populate Data): Wrote 2 batches with 3 rows each"</span><span class="p">);</span> |
| <span class="p">}</span> |
| </pre></div> |
| </div> |
| <div class="highlight-shell notranslate"><div class="highlight"><pre><span></span>C2:<span class="w"> </span>Client<span class="w"> </span><span class="o">(</span>Populate<span class="w"> </span>Data<span class="o">)</span>:<span class="w"> </span>Wrote<span class="w"> </span><span class="m">2</span><span class="w"> </span>batches<span class="w"> </span>with<span class="w"> </span><span class="m">3</span><span class="w"> </span>rows<span class="w"> </span>each |
| </pre></div> |
| </div> |
| </section> |
| <section id="get-metadata"> |
| <h3><a class="toc-backref" href="#id7" role="doc-backlink">Get Metadata</a><a class="headerlink" href="#get-metadata" title="Link to this heading">¶</a></h3> |
| <p>Once we do so, we can retrieve the metadata for that dataset.</p> |
| <div class="highlight-java notranslate"><div class="highlight"><pre><span></span><span class="c1">// Server</span> |
| <span class="kd">public</span><span class="w"> </span><span class="n">FlightInfo</span><span class="w"> </span><span class="nf">getFlightInfo</span><span class="p">(</span><span class="n">CallContext</span><span class="w"> </span><span class="n">context</span><span class="p">,</span><span class="w"> </span><span class="n">FlightDescriptor</span><span class="w"> </span><span class="n">descriptor</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">FlightEndpoint</span><span class="w"> </span><span class="n">flightEndpoint</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">FlightEndpoint</span><span class="p">(</span> |
| <span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">Ticket</span><span class="p">(</span><span class="n">descriptor</span><span class="p">.</span><span class="na">getPath</span><span class="p">().</span><span class="na">get</span><span class="p">(</span><span class="mi">0</span><span class="p">).</span><span class="na">getBytes</span><span class="p">(</span><span class="n">StandardCharsets</span><span class="p">.</span><span class="na">UTF_8</span><span class="p">)),</span><span class="w"> </span><span class="n">location</span><span class="p">);</span> |
| <span class="w"> </span><span class="k">return</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">FlightInfo</span><span class="p">(</span> |
| <span class="w"> </span><span class="n">datasets</span><span class="p">.</span><span class="na">get</span><span class="p">(</span><span class="n">descriptor</span><span class="p">).</span><span class="na">getSchema</span><span class="p">(),</span> |
| <span class="w"> </span><span class="n">descriptor</span><span class="p">,</span> |
| <span class="w"> </span><span class="n">Collections</span><span class="p">.</span><span class="na">singletonList</span><span class="p">(</span><span class="n">flightEndpoint</span><span class="p">),</span> |
| <span class="w"> </span><span class="cm">/*bytes=*/</span><span class="o">-</span><span class="mi">1</span><span class="p">,</span> |
| <span class="w"> </span><span class="n">datasets</span><span class="p">.</span><span class="na">get</span><span class="p">(</span><span class="n">descriptor</span><span class="p">).</span><span class="na">getRows</span><span class="p">()</span> |
| <span class="w"> </span><span class="p">);</span> |
| <span class="p">}</span> |
| |
| <span class="c1">// Client</span> |
| <span class="n">FlightInfo</span><span class="w"> </span><span class="n">flightInfo</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">flightClient</span><span class="p">.</span><span class="na">getInfo</span><span class="p">(</span><span class="n">FlightDescriptor</span><span class="p">.</span><span class="na">path</span><span class="p">(</span><span class="s">"profiles"</span><span class="p">));</span> |
| <span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"C3: Client (Get Metadata): "</span><span class="w"> </span><span class="o">+</span><span class="w"> </span><span class="n">flightInfo</span><span class="p">);</span> |
| </pre></div> |
| </div> |
| <div class="highlight-shell notranslate"><div class="highlight"><pre><span></span>C3:<span class="w"> </span>Client<span class="w"> </span><span class="o">(</span>Get<span class="w"> </span>Metadata<span class="o">)</span>:<span class="w"> </span>FlightInfo<span class="o">{</span><span class="nv">schema</span><span class="o">=</span>Schema<name:<span class="w"> </span>Utf8>,<span class="w"> </span><span class="nv">descriptor</span><span class="o">=</span>profiles,<span class="w"> </span><span class="nv">endpoints</span><span class="o">=[</span>FlightEndpoint<span class="o">{</span><span class="nv">locations</span><span class="o">=[</span>Location<span class="o">{</span><span class="nv">uri</span><span class="o">=</span>grpc+tcp://0.0.0.0:33333<span class="o">}]</span>,<span class="w"> </span><span class="nv">ticket</span><span class="o">=</span>org.apache.arrow.flight.Ticket@58871b0a,<span class="w"> </span><span class="nv">expirationTime</span><span class="o">=(</span>none<span class="o">)}]</span>,<span class="w"> </span><span class="nv">bytes</span><span class="o">=</span>-1,<span class="w"> </span><span class="nv">records</span><span class="o">=</span><span class="m">6</span><span class="o">}</span> |
| </pre></div> |
| </div> |
| </section> |
| <section id="get-data"> |
| <h3><a class="toc-backref" href="#id8" role="doc-backlink">Get Data</a><a class="headerlink" href="#get-data" title="Link to this heading">¶</a></h3> |
| <p>And get the data back:</p> |
| <div class="highlight-java notranslate"><div class="highlight"><pre><span></span><span class="c1">// Server</span> |
| <span class="kd">public</span><span class="w"> </span><span class="kt">void</span><span class="w"> </span><span class="nf">getStream</span><span class="p">(</span><span class="n">CallContext</span><span class="w"> </span><span class="n">context</span><span class="p">,</span><span class="w"> </span><span class="n">Ticket</span><span class="w"> </span><span class="n">ticket</span><span class="p">,</span><span class="w"> </span><span class="n">ServerStreamListener</span><span class="w"> </span><span class="n">listener</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">FlightDescriptor</span><span class="w"> </span><span class="n">flightDescriptor</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">FlightDescriptor</span><span class="p">.</span><span class="na">path</span><span class="p">(</span> |
| <span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">String</span><span class="p">(</span><span class="n">ticket</span><span class="p">.</span><span class="na">getBytes</span><span class="p">(),</span><span class="w"> </span><span class="n">StandardCharsets</span><span class="p">.</span><span class="na">UTF_8</span><span class="p">));</span> |
| <span class="w"> </span><span class="n">Dataset</span><span class="w"> </span><span class="n">dataset</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">this</span><span class="p">.</span><span class="na">datasets</span><span class="p">.</span><span class="na">get</span><span class="p">(</span><span class="n">flightDescriptor</span><span class="p">);</span> |
| <span class="w"> </span><span class="k">if</span><span class="w"> </span><span class="p">(</span><span class="n">dataset</span><span class="w"> </span><span class="o">==</span><span class="w"> </span><span class="kc">null</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="k">throw</span><span class="w"> </span><span class="n">CallStatus</span><span class="p">.</span><span class="na">NOT_FOUND</span><span class="p">.</span><span class="na">withDescription</span><span class="p">(</span><span class="s">"Unknown descriptor"</span><span class="p">).</span><span class="na">toRuntimeException</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">}</span><span class="w"> </span><span class="k">else</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">VectorSchemaRoot</span><span class="w"> </span><span class="n">vectorSchemaRoot</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">VectorSchemaRoot</span><span class="p">.</span><span class="na">create</span><span class="p">(</span> |
| <span class="w"> </span><span class="k">this</span><span class="p">.</span><span class="na">datasets</span><span class="p">.</span><span class="na">get</span><span class="p">(</span><span class="n">flightDescriptor</span><span class="p">).</span><span class="na">getSchema</span><span class="p">(),</span><span class="w"> </span><span class="n">allocator</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">start</span><span class="p">(</span><span class="n">vectorSchemaRoot</span><span class="p">);</span> |
| <span class="w"> </span><span class="k">for</span><span class="w"> </span><span class="p">(</span><span class="n">ArrowRecordBatch</span><span class="w"> </span><span class="n">arrowRecordBatch</span><span class="w"> </span><span class="p">:</span><span class="w"> </span><span class="k">this</span><span class="p">.</span><span class="na">datasets</span><span class="p">.</span><span class="na">get</span><span class="p">(</span><span class="n">flightDescriptor</span><span class="p">).</span><span class="na">getBatches</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">VectorLoader</span><span class="w"> </span><span class="n">loader</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">VectorLoader</span><span class="p">(</span><span class="n">vectorSchemaRoot</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">loader</span><span class="p">.</span><span class="na">load</span><span class="p">(</span><span class="n">arrowRecordBatch</span><span class="p">.</span><span class="na">cloneWithTransfer</span><span class="p">(</span><span class="n">allocator</span><span class="p">));</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">putNext</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">completed</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="p">}</span> |
| |
| <span class="c1">// Client</span> |
| <span class="k">try</span><span class="p">(</span><span class="n">FlightStream</span><span class="w"> </span><span class="n">flightStream</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">flightClient</span><span class="p">.</span><span class="na">getStream</span><span class="p">(</span><span class="n">flightInfo</span><span class="p">.</span><span class="na">getEndpoints</span><span class="p">().</span><span class="na">get</span><span class="p">(</span><span class="mi">0</span><span class="p">).</span><span class="na">getTicket</span><span class="p">()))</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="kt">int</span><span class="w"> </span><span class="n">batch</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="mi">0</span><span class="p">;</span> |
| <span class="w"> </span><span class="k">try</span><span class="w"> </span><span class="p">(</span><span class="n">VectorSchemaRoot</span><span class="w"> </span><span class="n">vectorSchemaRootReceived</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">flightStream</span><span class="p">.</span><span class="na">getRoot</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"C4: Client (Get Stream):"</span><span class="p">);</span> |
| <span class="w"> </span><span class="k">while</span><span class="w"> </span><span class="p">(</span><span class="n">flightStream</span><span class="p">.</span><span class="na">next</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">batch</span><span class="o">++</span><span class="p">;</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"Client Received batch #"</span><span class="w"> </span><span class="o">+</span><span class="w"> </span><span class="n">batch</span><span class="w"> </span><span class="o">+</span><span class="w"> </span><span class="s">", Data:"</span><span class="p">);</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">print</span><span class="p">(</span><span class="n">vectorSchemaRootReceived</span><span class="p">.</span><span class="na">contentToTSVString</span><span class="p">());</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="p">}</span><span class="w"> </span><span class="k">catch</span><span class="w"> </span><span class="p">(</span><span class="n">Exception</span><span class="w"> </span><span class="n">e</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">e</span><span class="p">.</span><span class="na">printStackTrace</span><span class="p">();</span> |
| <span class="p">}</span> |
| </pre></div> |
| </div> |
| <div class="highlight-shell notranslate"><div class="highlight"><pre><span></span>C4:<span class="w"> </span>Client<span class="w"> </span><span class="o">(</span>Get<span class="w"> </span>Stream<span class="o">)</span>: |
| Client<span class="w"> </span>Received<span class="w"> </span>batch<span class="w"> </span><span class="c1">#1, Data:</span> |
| name |
| Ronald |
| David |
| Francisco |
| Client<span class="w"> </span>Received<span class="w"> </span>batch<span class="w"> </span><span class="c1">#2, Data:</span> |
| name |
| Manuel |
| Felipe |
| JJ |
| </pre></div> |
| </div> |
| </section> |
| <section id="delete-data"> |
| <h3><a class="toc-backref" href="#id9" role="doc-backlink">Delete data</a><a class="headerlink" href="#delete-data" title="Link to this heading">¶</a></h3> |
| <p>Then, we’ll delete the dataset:</p> |
| <div class="highlight-java notranslate"><div class="highlight"><pre><span></span><span class="c1">// Server</span> |
| <span class="kd">public</span><span class="w"> </span><span class="kt">void</span><span class="w"> </span><span class="nf">doAction</span><span class="p">(</span><span class="n">CallContext</span><span class="w"> </span><span class="n">context</span><span class="p">,</span><span class="w"> </span><span class="n">Action</span><span class="w"> </span><span class="n">action</span><span class="p">,</span><span class="w"> </span><span class="n">StreamListener</span><span class="o"><</span><span class="n">Result</span><span class="o">></span><span class="w"> </span><span class="n">listener</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">FlightDescriptor</span><span class="w"> </span><span class="n">flightDescriptor</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">FlightDescriptor</span><span class="p">.</span><span class="na">path</span><span class="p">(</span> |
| <span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">String</span><span class="p">(</span><span class="n">action</span><span class="p">.</span><span class="na">getBody</span><span class="p">(),</span><span class="w"> </span><span class="n">StandardCharsets</span><span class="p">.</span><span class="na">UTF_8</span><span class="p">));</span> |
| <span class="w"> </span><span class="k">switch</span><span class="w"> </span><span class="p">(</span><span class="n">action</span><span class="p">.</span><span class="na">getType</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="k">case</span><span class="w"> </span><span class="s">"DELETE"</span><span class="p">:</span> |
| <span class="w"> </span><span class="k">if</span><span class="w"> </span><span class="p">(</span><span class="n">datasets</span><span class="p">.</span><span class="na">remove</span><span class="p">(</span><span class="n">flightDescriptor</span><span class="p">)</span><span class="w"> </span><span class="o">!=</span><span class="w"> </span><span class="kc">null</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">Result</span><span class="w"> </span><span class="n">result</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">Result</span><span class="p">(</span><span class="s">"Delete completed"</span><span class="p">.</span><span class="na">getBytes</span><span class="p">(</span><span class="n">StandardCharsets</span><span class="p">.</span><span class="na">UTF_8</span><span class="p">));</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">onNext</span><span class="p">(</span><span class="n">result</span><span class="p">);</span> |
| <span class="w"> </span><span class="p">}</span><span class="w"> </span><span class="k">else</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">Result</span><span class="w"> </span><span class="n">result</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">Result</span><span class="p">(</span><span class="s">"Delete not completed. Reason: Key did not exist."</span> |
| <span class="w"> </span><span class="p">.</span><span class="na">getBytes</span><span class="p">(</span><span class="n">StandardCharsets</span><span class="p">.</span><span class="na">UTF_8</span><span class="p">));</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">onNext</span><span class="p">(</span><span class="n">result</span><span class="p">);</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">onCompleted</span><span class="p">();</span> |
| <span class="w"> </span><span class="p">}</span> |
| <span class="p">}</span> |
| |
| <span class="c1">// Client</span> |
| <span class="n">Iterator</span><span class="o"><</span><span class="n">Result</span><span class="o">></span><span class="w"> </span><span class="n">deleteActionResult</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">flightClient</span><span class="p">.</span><span class="na">doAction</span><span class="p">(</span><span class="k">new</span><span class="w"> </span><span class="n">Action</span><span class="p">(</span><span class="s">"DELETE"</span><span class="p">,</span> |
| <span class="w"> </span><span class="n">FlightDescriptor</span><span class="p">.</span><span class="na">path</span><span class="p">(</span><span class="s">"profiles"</span><span class="p">).</span><span class="na">getPath</span><span class="p">().</span><span class="na">get</span><span class="p">(</span><span class="mi">0</span><span class="p">).</span><span class="na">getBytes</span><span class="p">(</span><span class="n">StandardCharsets</span><span class="p">.</span><span class="na">UTF_8</span><span class="p">)));</span> |
| <span class="k">while</span><span class="w"> </span><span class="p">(</span><span class="n">deleteActionResult</span><span class="p">.</span><span class="na">hasNext</span><span class="p">())</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">Result</span><span class="w"> </span><span class="n">result</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">deleteActionResult</span><span class="p">.</span><span class="na">next</span><span class="p">();</span> |
| <span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"C6: Client (Do Delete Action): "</span><span class="w"> </span><span class="o">+</span> |
| <span class="w"> </span><span class="k">new</span><span class="w"> </span><span class="n">String</span><span class="p">(</span><span class="n">result</span><span class="p">.</span><span class="na">getBody</span><span class="p">(),</span><span class="w"> </span><span class="n">StandardCharsets</span><span class="p">.</span><span class="na">UTF_8</span><span class="p">));</span> |
| <span class="p">}</span> |
| </pre></div> |
| </div> |
| <div class="highlight-shell notranslate"><div class="highlight"><pre><span></span>C6:<span class="w"> </span>Client<span class="w"> </span><span class="o">(</span>Do<span class="w"> </span>Delete<span class="w"> </span>Action<span class="o">)</span>:<span class="w"> </span>Delete<span class="w"> </span>completed |
| </pre></div> |
| </div> |
| </section> |
| <section id="validate-delete-data"> |
| <h3><a class="toc-backref" href="#id10" role="doc-backlink">Validate Delete Data</a><a class="headerlink" href="#validate-delete-data" title="Link to this heading">¶</a></h3> |
| <p>And confirm that it’s been deleted:</p> |
| <div class="highlight-java notranslate"><div class="highlight"><pre><span></span><span class="c1">// Server</span> |
| <span class="kd">public</span><span class="w"> </span><span class="kt">void</span><span class="w"> </span><span class="nf">listFlights</span><span class="p">(</span><span class="n">CallContext</span><span class="w"> </span><span class="n">context</span><span class="p">,</span><span class="w"> </span><span class="n">Criteria</span><span class="w"> </span><span class="n">criteria</span><span class="p">,</span><span class="w"> </span><span class="n">StreamListener</span><span class="o"><</span><span class="n">FlightInfo</span><span class="o">></span><span class="w"> </span><span class="n">listener</span><span class="p">)</span><span class="w"> </span><span class="p">{</span> |
| <span class="w"> </span><span class="n">datasets</span><span class="p">.</span><span class="na">forEach</span><span class="p">((</span><span class="n">k</span><span class="p">,</span><span class="w"> </span><span class="n">v</span><span class="p">)</span><span class="w"> </span><span class="o">-></span><span class="w"> </span><span class="p">{</span><span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">onNext</span><span class="p">(</span><span class="n">getFlightInfo</span><span class="p">(</span><span class="kc">null</span><span class="p">,</span><span class="w"> </span><span class="n">k</span><span class="p">));</span><span class="w"> </span><span class="p">});</span> |
| <span class="w"> </span><span class="n">listener</span><span class="p">.</span><span class="na">onCompleted</span><span class="p">();</span> |
| <span class="p">}</span> |
| |
| <span class="c1">// Client</span> |
| <span class="n">Iterable</span><span class="o"><</span><span class="n">FlightInfo</span><span class="o">></span><span class="w"> </span><span class="n">flightInfos</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">flightClient</span><span class="p">.</span><span class="na">listFlights</span><span class="p">(</span><span class="n">Criteria</span><span class="p">.</span><span class="na">ALL</span><span class="p">);</span> |
| <span class="n">flightInfos</span><span class="p">.</span><span class="na">forEach</span><span class="p">(</span><span class="n">t</span><span class="w"> </span><span class="o">-></span><span class="w"> </span><span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="n">t</span><span class="p">));</span> |
| <span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"C7: Client (List Flights Info): After delete - No records"</span><span class="p">);</span> |
| </pre></div> |
| </div> |
| <div class="highlight-shell notranslate"><div class="highlight"><pre><span></span>C7:<span class="w"> </span>Client<span class="w"> </span><span class="o">(</span>List<span class="w"> </span>Flights<span class="w"> </span>Info<span class="o">)</span>:<span class="w"> </span>After<span class="w"> </span>delete<span class="w"> </span>-<span class="w"> </span>No<span class="w"> </span>records |
| </pre></div> |
| </div> |
| </section> |
| <section id="stop-flight-server"> |
| <h3><a class="toc-backref" href="#id11" role="doc-backlink">Stop Flight Server</a><a class="headerlink" href="#stop-flight-server" title="Link to this heading">¶</a></h3> |
| <div class="highlight-java notranslate"><div class="highlight"><pre><span></span><span class="c1">// Server</span> |
| <span class="n">flightServer</span><span class="p">.</span><span class="na">shutdown</span><span class="p">();</span> |
| <span class="n">System</span><span class="p">.</span><span class="na">out</span><span class="p">.</span><span class="na">println</span><span class="p">(</span><span class="s">"C8: Server shut down successfully"</span><span class="p">);</span> |
| </pre></div> |
| </div> |
| <div class="highlight-shell notranslate"><div class="highlight"><pre><span></span>C8:<span class="w"> </span>Server<span class="w"> </span>shut<span class="w"> </span>down<span class="w"> </span>successfully |
| </pre></div> |
| </div> |
| <p><span class="target" id="arrow-flight-rpc">Arrow Flight RPC</span>: <a class="reference external" href="https://arrow.apache.org/docs/format/Flight.html">https://arrow.apache.org/docs/format/Flight.html</a></p> |
| </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="create.html">Creating Arrow Objects</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="schema.html">Working with Schema</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="io.html">Reading and writing data</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-key-value-storage-service-with-arrow-flight">Simple Key-Value Storage Service with Arrow Flight</a></li> |
| </ul> |
| </li> |
| <li class="toctree-l1"><a class="reference internal" href="dataset.html">Dataset</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="substrait.html">Substrait</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="data.html">Data manipulation</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="avro.html">Avro</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="jdbc.html">Arrow JDBC Adapter</a></li> |
| </ul> |
| |
| |
| <hr /> |
| <ul> |
| |
| <li class="toctree-l1"><a href="https://arrow.apache.org/docs/java/index.html">User Guide</a></li> |
| |
| <li class="toctree-l1"><a href="https://arrow.apache.org/docs/java/reference/index.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="io.html" title="previous chapter">Reading and writing data</a></li> |
| <li>Next: <a href="dataset.html" title="next chapter">Dataset</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"> |
| ©2022, Apache Software Foundation. |
| |
| | |
| Powered by <a href="https://www.sphinx-doc.org/">Sphinx 7.2.6</a> |
| & <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> |