Browse Source

ESQL: add Arrow dataframes output format (#109873)

Initial support for Apache Arrow's streaming format as a response for ES|QL. It triggers based on the Accept header or the format request parameter.

Arrow has implementations in every mainstream language and is a backend of the Python Pandas library, which is extremely popular among data scientists and data analysts. Arrow's streaming format has also become the de facto standard for dataframe interchange. It is an efficient binary format that allows zero-cost deserialization by adding data access wrappers on top of memory buffers received from the network.

This PR builds on the experiment made by @nik9000 in PR #104877

Features/limitations:
- all ES|QL data types are supported
- multi-valued fields are not supported
- fields of type _source are output as JSON text in a varchar array. In a future iteration we may want to offer the choice of the more efficient CBOR and SMILE formats.

Technical details:

Arrow comes with its own memory management to handle vectors with direct memory, reference counting, etc. We don't want to use this as it conflicts with Elasticsearch's own memory management.

We therefore use the Arrow library only for the metadata objects describing the dataframe schema and the structure of the streaming format. The Arrow vector data is produced directly from ES|QL blocks.

---------

Co-authored-by: Nik Everett <nik9000@gmail.com>
Sylvain Wallez 1 year ago
parent
commit
e78bdc953a
30 changed files with 4932 additions and 1 deletions
  1. 5 0
      docs/changelog/109873.yaml
  2. 3 0
      docs/reference/esql/esql-rest.asciidoc
  3. 30 0
      gradle/verification-metadata.xml
  4. 24 0
      server/src/main/java/org/elasticsearch/common/io/stream/RecyclerBytesStreamOutput.java
  5. 22 0
      server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java
  6. 61 0
      x-pack/plugin/esql/arrow/build.gradle
  7. 2261 0
      x-pack/plugin/esql/arrow/licenses/arrow-LICENSE.txt
  8. 84 0
      x-pack/plugin/esql/arrow/licenses/arrow-NOTICE.txt
  9. 22 0
      x-pack/plugin/esql/arrow/licenses/checker-qual-LICENSE.txt
  10. 0 0
      x-pack/plugin/esql/arrow/licenses/checker-qual-NOTICE.txt
  11. 202 0
      x-pack/plugin/esql/arrow/licenses/flatbuffers-java-LICENSE.txt
  12. 0 0
      x-pack/plugin/esql/arrow/licenses/flatbuffers-java-NOTICE.txt
  13. 202 0
      x-pack/plugin/esql/arrow/licenses/jackson-LICENSE.txt
  14. 0 0
      x-pack/plugin/esql/arrow/licenses/jackson-NOTICE.txt
  15. 21 0
      x-pack/plugin/esql/arrow/licenses/slf4j-LICENSE.txt
  16. 0 0
      x-pack/plugin/esql/arrow/licenses/slf4j-NOTICE.txt
  17. 69 0
      x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/AllocationManagerShim.java
  18. 35 0
      x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/ArrowFormat.java
  19. 379 0
      x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/ArrowResponse.java
  20. 452 0
      x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/BlockConverter.java
  21. 80 0
      x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/ValueConversions.java
  22. 600 0
      x-pack/plugin/esql/arrow/src/test/java/org/elasticsearch/xpack/esql/arrow/ArrowResponseTests.java
  23. 84 0
      x-pack/plugin/esql/arrow/src/test/java/org/elasticsearch/xpack/esql/arrow/ValueConversionsTests.java
  24. 13 0
      x-pack/plugin/esql/arrow/src/test/resources/plugin-security.policy
  25. 2 0
      x-pack/plugin/esql/build.gradle
  26. 14 0
      x-pack/plugin/esql/qa/server/single-node/build.gradle
  27. 242 0
      x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/ArrowFormatIT.java
  28. 11 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlResponseListener.java
  29. 2 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlMediaTypeParser.java
  30. 12 0
      x-pack/plugin/esql/src/main/plugin-metadata/plugin-security.policy

+ 5 - 0
docs/changelog/109873.yaml

@@ -0,0 +1,5 @@
+pr: 109873
+summary: "ESQL: add Arrow dataframes output format"
+area: ES|QL
+type: feature
+issues: []

+ 3 - 0
docs/reference/esql/esql-rest.asciidoc

@@ -111,6 +111,9 @@ s|Description
 |{wikipedia}/Smile_(data_interchange_format)[Smile] binary data format similar
 to CBOR
 
+|arrow
+|application/vnd.apache.arrow.stream
+|**Experimental.** https://arrow.apache.org/[Apache Arrow] dataframes, https://arrow.apache.org/docs/format/Columnar.html#ipc-streaming-format[IPC streaming format]
 |===
 
 The `csv` format accepts a formatting URL query attribute, `delimiter`, which

+ 30 - 0
gradle/verification-metadata.xml

@@ -581,6 +581,11 @@
             <sha256 value="baf7d6ea97ce606c53e11b6854ba5f2ce7ef5c24dddf0afa18d1260bd25b002c" origin="Generated by Gradle"/>
          </artifact>
       </component>
+      <component group="com.google.flatbuffers" name="flatbuffers-java" version="23.5.26">
+         <artifact name="flatbuffers-java-23.5.26.jar">
+            <sha256 value="8d10cac2ea9878896077ba437d76fdb1b9a07f55a863c560bb8a024b04103f8b" origin="Generated by Gradle"/>
+         </artifact>
+      </component>
       <component group="com.google.googlejavaformat" name="google-java-format" version="1.16.0">
          <artifact name="google-java-format-1.16.0.jar">
             <sha256 value="0cff5d0230ba20d538f3f70b2aa68bd33f9fdc69768cde07337c563c23eb7c43" origin="Generated by Gradle"/>
@@ -1841,6 +1846,26 @@
             <sha256 value="cd7695b3bfb6964ab71b6a0b31dad60005ae77fe502132364679aacf08f77970" origin="Generated by Gradle"/>
          </artifact>
       </component>
+      <component group="org.apache.arrow" name="arrow-format" version="16.1.0">
+         <artifact name="arrow-format-16.1.0.jar">
+            <sha256 value="ad97e0fc72e193b1de3cbce4818d1ff16e81673fd523d001e8d2774bde40ee6c" origin="Generated by Gradle"/>
+         </artifact>
+      </component>
+      <component group="org.apache.arrow" name="arrow-memory-core" version="16.1.0">
+         <artifact name="arrow-memory-core-16.1.0.jar">
+            <sha256 value="da7af1a1a899bd5a1b6c71284243b9f3c0e1098f0cb10cd7be4b8b455ced79dd" origin="Generated by Gradle"/>
+         </artifact>
+      </component>
+      <component group="org.apache.arrow" name="arrow-memory-unsafe" version="16.1.0">
+         <artifact name="arrow-memory-unsafe-16.1.0.jar">
+            <sha256 value="6534eded25f2c30593416a294c1047f0b017baa9906d98f6f3270737b076c745" origin="Generated by Gradle"/>
+         </artifact>
+      </component>
+      <component group="org.apache.arrow" name="arrow-vector" version="16.1.0">
+         <artifact name="arrow-vector-16.1.0.jar">
+            <sha256 value="c5837b3aa24dfd93759f57bc5759b9a8fbb5bf3912d55994d70cabb904436aab" origin="Generated by Gradle"/>
+         </artifact>
+      </component>
       <component group="org.apache.avro" name="avro" version="1.7.4">
          <artifact name="avro-1.7.4.jar">
             <sha256 value="a01d26e9a5ed0754e8c88dbb373fba896c57df0a0c424185767a3857855bb222" origin="Generated by Gradle"/>
@@ -3177,6 +3202,11 @@
             <sha256 value="e316255bbfcd9fe50d165314b85abb2b33cb2a66a93c491db648e498a82c2de1" origin="Generated by Gradle"/>
          </artifact>
       </component>
+      <component group="org.checkerframework" name="checker-qual" version="3.42.0">
+         <artifact name="checker-qual-3.42.0.jar">
+            <sha256 value="ccaedd33af0b7894d9f2f3b644f4d19e43928e32902e61ac4d10777830f5aac7" origin="Generated by Gradle"/>
+         </artifact>
+      </component>
       <component group="org.checkerframework" name="checker-qual" version="3.5.0">
          <artifact name="checker-qual-3.5.0.jar">
             <sha256 value="729990b3f18a95606fc2573836b6958bcdb44cb52bfbd1b7aa9c339cff35a5a4" origin="Generated by Gradle"/>

+ 24 - 0
server/src/main/java/org/elasticsearch/common/io/stream/RecyclerBytesStreamOutput.java

@@ -32,7 +32,9 @@ import java.util.Objects;
 public class RecyclerBytesStreamOutput extends BytesStream implements Releasable {
 
     static final VarHandle VH_BE_INT = MethodHandles.byteArrayViewVarHandle(int[].class, ByteOrder.BIG_ENDIAN);
+    static final VarHandle VH_LE_INT = MethodHandles.byteArrayViewVarHandle(int[].class, ByteOrder.LITTLE_ENDIAN);
     static final VarHandle VH_BE_LONG = MethodHandles.byteArrayViewVarHandle(long[].class, ByteOrder.BIG_ENDIAN);
+    static final VarHandle VH_LE_LONG = MethodHandles.byteArrayViewVarHandle(long[].class, ByteOrder.LITTLE_ENDIAN);
 
     private final ArrayList<Recycler.V<BytesRef>> pages = new ArrayList<>();
     private final Recycler<BytesRef> recycler;
@@ -106,6 +108,17 @@ public class RecyclerBytesStreamOutput extends BytesStream implements Releasable
         }
     }
 
+    @Override
+    public void writeIntLE(int i) throws IOException {
+        if (4 > (pageSize - currentPageOffset)) {
+            super.writeIntLE(i);
+        } else {
+            BytesRef currentPage = pages.get(pageIndex).v();
+            VH_LE_INT.set(currentPage.bytes, currentPage.offset + currentPageOffset, i);
+            currentPageOffset += 4;
+        }
+    }
+
     @Override
     public void writeLong(long i) throws IOException {
         if (8 > (pageSize - currentPageOffset)) {
@@ -117,6 +130,17 @@ public class RecyclerBytesStreamOutput extends BytesStream implements Releasable
         }
     }
 
+    @Override
+    public void writeLongLE(long i) throws IOException {
+        if (8 > (pageSize - currentPageOffset)) {
+            super.writeLongLE(i);
+        } else {
+            BytesRef currentPage = pages.get(pageIndex).v();
+            VH_LE_LONG.set(currentPage.bytes, currentPage.offset + currentPageOffset, i);
+            currentPageOffset += 8;
+        }
+    }
+
     @Override
     public void writeWithSizePrefix(Writeable writeable) throws IOException {
         // TODO: do this without copying the bytes from tmp by calling writeBytes and just use the pages in tmp directly through

+ 22 - 0
server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java

@@ -190,6 +190,15 @@ public abstract class StreamOutput extends OutputStream {
         writeBytes(buffer, 0, 4);
     }
 
+    /**
+     * Writes an int as four bytes, least significant bytes first.
+     */
+    public void writeIntLE(int i) throws IOException {
+        final byte[] buffer = scratch.get();
+        ByteUtils.writeIntLE(i, buffer, 0);
+        writeBytes(buffer, 0, 4);
+    }
+
     /**
      * Writes an int in a variable-length format.  Writes between one and
      * five bytes.  Smaller values take fewer bytes.  Negative numbers
@@ -243,6 +252,15 @@ public abstract class StreamOutput extends OutputStream {
         writeBytes(buffer, 0, 8);
     }
 
+    /**
+     * Writes a long as eight bytes.
+     */
+    public void writeLongLE(long i) throws IOException {
+        final byte[] buffer = scratch.get();
+        ByteUtils.writeLongLE(i, buffer, 0);
+        writeBytes(buffer, 0, 8);
+    }
+
     /**
      * Writes a non-negative long in a variable-length format. Writes between one and ten bytes. Smaller values take fewer bytes. Negative
      * numbers use ten bytes and trip assertions (if running in tests) so prefer {@link #writeLong(long)} or {@link #writeZLong(long)} for
@@ -442,6 +460,10 @@ public abstract class StreamOutput extends OutputStream {
         writeLong(Double.doubleToLongBits(v));
     }
 
+    public void writeDoubleLE(double v) throws IOException {
+        writeLongLE(Double.doubleToLongBits(v));
+    }
+
     public void writeOptionalDouble(@Nullable Double v) throws IOException {
         if (v == null) {
             writeBoolean(false);

+ 61 - 0
x-pack/plugin/esql/arrow/build.gradle

@@ -0,0 +1,61 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+apply plugin: 'elasticsearch.build'
+
+dependencies {
+  compileOnly project(':server')
+  compileOnly project(':x-pack:plugin:esql:compute')
+  compileOnly project(':x-pack:plugin:esql-core')
+  compileOnly project(':x-pack:plugin:mapper-version')
+  implementation('org.apache.arrow:arrow-vector:16.1.0')
+  implementation('org.apache.arrow:arrow-format:16.1.0')
+  implementation('org.apache.arrow:arrow-memory-core:16.1.0')
+  implementation('org.checkerframework:checker-qual:3.42.0')
+  implementation('com.google.flatbuffers:flatbuffers-java:23.5.26')
+  // Needed for the json arrow serialization, and loaded even if we don't use it.
+  implementation("com.fasterxml.jackson.core:jackson-annotations:${versions.jackson}")
+  implementation("com.fasterxml.jackson.core:jackson-core:${versions.jackson}")
+  implementation("com.fasterxml.jackson.core:jackson-databind:${versions.jackson}")
+  implementation("org.slf4j:slf4j-api:${versions.slf4j}")
+  runtimeOnly "org.slf4j:slf4j-nop:${versions.slf4j}"
+
+  testImplementation project(':test:framework')
+  testImplementation('org.apache.arrow:arrow-memory-unsafe:16.1.0')
+}
+
+tasks.named("dependencyLicenses").configure {
+  mapping from: /jackson-.*/, to: 'jackson'
+  mapping from: /arrow-.*/, to: 'arrow'
+  mapping from: /slf4j-.*/, to: 'slf4j'
+}
+
+tasks.named("thirdPartyAudit").configure {
+  ignoreViolations(
+    // uses sun.misc.Unsafe. Only used in tests.
+    'org.apache.arrow.memory.util.hash.SimpleHasher',
+    'org.apache.arrow.memory.util.hash.MurmurHasher',
+    'org.apache.arrow.memory.util.MemoryUtil',
+    'org.apache.arrow.memory.util.MemoryUtil$1',
+    'org.apache.arrow.vector.DecimalVector',
+    'org.apache.arrow.vector.BaseFixedWidthVector',
+    'org.apache.arrow.vector.util.DecimalUtility',
+    'org.apache.arrow.vector.Decimal256Vector',
+    'org.apache.arrow.vector.util.VectorAppender',
+    'org.apache.arrow.memory.ArrowBuf',
+    'org.apache.arrow.vector.BitVectorHelper',
+    'org.apache.arrow.memory.util.ByteFunctionHelpers',
+  )
+  ignoreMissingClasses(
+    'org.apache.commons.codec.binary.Hex'
+  )
+}
+
+test {
+  jvmArgs('--add-opens=java.base/java.nio=ALL-UNNAMED')
+}

+ 2261 - 0
x-pack/plugin/esql/arrow/licenses/arrow-LICENSE.txt

@@ -0,0 +1,2261 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+
+--------------------------------------------------------------------------------
+
+src/arrow/util (some portions): Apache 2.0, and 3-clause BSD
+
+Some portions of this module are derived from code in the Chromium project,
+copyright (c) Google inc and (c) The Chromium Authors and licensed under the
+Apache 2.0 License or the under the 3-clause BSD license:
+
+  Copyright (c) 2013 The Chromium Authors. All rights reserved.
+
+  Redistribution and use in source and binary forms, with or without
+  modification, are permitted provided that the following conditions are
+  met:
+
+     * Redistributions of source code must retain the above copyright
+  notice, this list of conditions and the following disclaimer.
+     * Redistributions in binary form must reproduce the above
+  copyright notice, this list of conditions and the following disclaimer
+  in the documentation and/or other materials provided with the
+  distribution.
+     * Neither the name of Google Inc. nor the names of its
+  contributors may be used to endorse or promote products derived from
+  this software without specific prior written permission.
+
+  THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+  "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+  LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+  A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+  OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+  SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+  LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+  DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+  THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+  (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+  OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+This project includes code from Daniel Lemire's FrameOfReference project.
+
+https://github.com/lemire/FrameOfReference/blob/6ccaf9e97160f9a3b299e23a8ef739e711ef0c71/src/bpacking.cpp
+https://github.com/lemire/FrameOfReference/blob/146948b6058a976bc7767262ad3a2ce201486b93/scripts/turbopacking64.py
+
+Copyright: 2013 Daniel Lemire
+Home page: http://lemire.me/en/
+Project page: https://github.com/lemire/FrameOfReference
+License: Apache License Version 2.0 http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This project includes code from the TensorFlow project
+
+Copyright 2015 The TensorFlow Authors. All Rights Reserved.
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+--------------------------------------------------------------------------------
+
+This project includes code from the NumPy project.
+
+https://github.com/numpy/numpy/blob/e1f191c46f2eebd6cb892a4bfe14d9dd43a06c4e/numpy/core/src/multiarray/multiarraymodule.c#L2910
+
+https://github.com/numpy/numpy/blob/68fd82271b9ea5a9e50d4e761061dfcca851382a/numpy/core/src/multiarray/datetime.c
+
+Copyright (c) 2005-2017, NumPy Developers.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+       notice, this list of conditions and the following disclaimer.
+
+    * Redistributions in binary form must reproduce the above
+       copyright notice, this list of conditions and the following
+       disclaimer in the documentation and/or other materials provided
+       with the distribution.
+
+    * Neither the name of the NumPy Developers nor the names of any
+       contributors may be used to endorse or promote products derived
+       from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+This project includes code from the Boost project
+
+Boost Software License - Version 1.0 - August 17th, 2003
+
+Permission is hereby granted, free of charge, to any person or organization
+obtaining a copy of the software and accompanying documentation covered by
+this license (the "Software") to use, reproduce, display, distribute,
+execute, and transmit the Software, and to prepare derivative works of the
+Software, and to permit third-parties to whom the Software is furnished to
+do so, all subject to the following:
+
+The copyright notices in the Software and this entire statement, including
+the above license grant, this restriction and the following disclaimer,
+must be included in all copies of the Software, in whole or in part, and
+all derivative works of the Software, unless such copies or derivative
+works are solely in the form of machine-executable object code generated by
+a source language processor.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT
+SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE
+FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE,
+ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
+DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+This project includes code from the FlatBuffers project
+
+Copyright 2014 Google Inc.
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+--------------------------------------------------------------------------------
+
+This project includes code from the tslib project
+
+Copyright 2015 Microsoft Corporation. All rights reserved.
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+--------------------------------------------------------------------------------
+
+This project includes code from the jemalloc project
+
+https://github.com/jemalloc/jemalloc
+
+Copyright (C) 2002-2017 Jason Evans <jasone@canonware.com>.
+All rights reserved.
+Copyright (C) 2007-2012 Mozilla Foundation.  All rights reserved.
+Copyright (C) 2009-2017 Facebook, Inc.  All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+1. Redistributions of source code must retain the above copyright notice(s),
+   this list of conditions and the following disclaimer.
+2. Redistributions in binary form must reproduce the above copyright notice(s),
+   this list of conditions and the following disclaimer in the documentation
+   and/or other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDER(S) ``AS IS'' AND ANY EXPRESS
+OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
+MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.  IN NO
+EVENT SHALL THE COPYRIGHT HOLDER(S) BE LIABLE FOR ANY DIRECT, INDIRECT,
+INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE
+OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF
+ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+--------------------------------------------------------------------------------
+
+This project includes code from the Go project, BSD 3-clause license + PATENTS
+weak patent termination clause
+(https://github.com/golang/go/blob/master/PATENTS).
+
+Copyright (c) 2009 The Go Authors. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+   * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+   * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+   * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+This project includes code from the hs2client
+
+https://github.com/cloudera/hs2client
+
+Copyright 2016 Cloudera Inc.
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+--------------------------------------------------------------------------------
+
+The script ci/scripts/util_wait_for_it.sh has the following license
+
+Copyright (c) 2016 Giles Hall
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
+of the Software, and to permit persons to whom the Software is furnished to do
+so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+The script r/configure has the following license (MIT)
+
+Copyright (c) 2017, Jeroen Ooms and Jim Hester
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
+of the Software, and to permit persons to whom the Software is furnished to do
+so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+cpp/src/arrow/util/logging.cc, cpp/src/arrow/util/logging.h and
+cpp/src/arrow/util/logging-test.cc are adapted from
+Ray Project (https://github.com/ray-project/ray) (Apache 2.0).
+
+Copyright (c) 2016 Ray Project (https://github.com/ray-project/ray)
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+--------------------------------------------------------------------------------
+The files cpp/src/arrow/vendored/datetime/date.h, cpp/src/arrow/vendored/datetime/tz.h,
+cpp/src/arrow/vendored/datetime/tz_private.h, cpp/src/arrow/vendored/datetime/ios.h,
+cpp/src/arrow/vendored/datetime/ios.mm,
+cpp/src/arrow/vendored/datetime/tz.cpp are adapted from
+Howard Hinnant's date library (https://github.com/HowardHinnant/date)
+It is licensed under MIT license.
+
+The MIT License (MIT)
+Copyright (c) 2015, 2016, 2017 Howard Hinnant
+Copyright (c) 2016 Adrian Colomitchi
+Copyright (c) 2017 Florian Dang
+Copyright (c) 2017 Paul Thompson
+Copyright (c) 2018 Tomasz Kamiński
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+The file cpp/src/arrow/util/utf8.h includes code adapted from the page
+  https://bjoern.hoehrmann.de/utf-8/decoder/dfa/
+with the following license (MIT)
+
+Copyright (c) 2008-2009 Bjoern Hoehrmann <bjoern@hoehrmann.de>
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+The files in cpp/src/arrow/vendored/xxhash/ have the following license
+(BSD 2-Clause License)
+
+xxHash Library
+Copyright (c) 2012-2014, Yann Collet
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice, this
+  list of conditions and the following disclaimer in the documentation and/or
+  other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+You can contact the author at :
+- xxHash homepage: http://www.xxhash.com
+- xxHash source repository : https://github.com/Cyan4973/xxHash
+
+--------------------------------------------------------------------------------
+
+The files in cpp/src/arrow/vendored/double-conversion/ have the following license
+(BSD 3-Clause License)
+
+Copyright 2006-2011, the V8 project authors. All rights reserved.
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+      copyright notice, this list of conditions and the following
+      disclaimer in the documentation and/or other materials provided
+      with the distribution.
+    * Neither the name of Google Inc. nor the names of its
+      contributors may be used to endorse or promote products derived
+      from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+The files in cpp/src/arrow/vendored/uriparser/ have the following license
+(BSD 3-Clause License)
+
+uriparser - RFC 3986 URI parsing library
+
+Copyright (C) 2007, Weijia Song <songweijia@gmail.com>
+Copyright (C) 2007, Sebastian Pipping <sebastian@pipping.org>
+All rights reserved.
+
+Redistribution  and use in source and binary forms, with or without
+modification,  are permitted provided that the following conditions
+are met:
+
+    * Redistributions   of  source  code  must  retain  the   above
+      copyright  notice, this list of conditions and the  following
+      disclaimer.
+
+    * Redistributions  in  binary  form must  reproduce  the  above
+      copyright  notice, this list of conditions and the  following
+      disclaimer   in  the  documentation  and/or  other  materials
+      provided with the distribution.
+
+    * Neither  the name of the <ORGANIZATION> nor the names of  its
+      contributors  may  be  used to endorse  or  promote  products
+      derived  from  this software without specific  prior  written
+      permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS  IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT  NOT
+LIMITED  TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND  FITNESS
+FOR  A  PARTICULAR  PURPOSE ARE DISCLAIMED. IN NO EVENT  SHALL  THE
+COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT,
+INCIDENTAL,    SPECIAL,   EXEMPLARY,   OR   CONSEQUENTIAL   DAMAGES
+(INCLUDING,  BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES;  LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
+HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+STRICT  LIABILITY,  OR  TORT (INCLUDING  NEGLIGENCE  OR  OTHERWISE)
+ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
+OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+The files under dev/tasks/conda-recipes have the following license
+
+BSD 3-clause license
+Copyright (c) 2015-2018, conda-forge
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+1. Redistributions of source code must retain the above copyright notice, this
+   list of conditions and the following disclaimer.
+
+2. Redistributions in binary form must reproduce the above copyright notice,
+   this list of conditions and the following disclaimer in the documentation
+   and/or other materials provided with the distribution.
+
+3. Neither the name of the copyright holder nor the names of its contributors
+   may be used to endorse or promote products derived from this software without
+   specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR
+TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+The files in cpp/src/arrow/vendored/utfcpp/ have the following license
+
+Copyright 2006-2018 Nemanja Trifunovic
+
+Permission is hereby granted, free of charge, to any person or organization
+obtaining a copy of the software and accompanying documentation covered by
+this license (the "Software") to use, reproduce, display, distribute,
+execute, and transmit the Software, and to prepare derivative works of the
+Software, and to permit third-parties to whom the Software is furnished to
+do so, all subject to the following:
+
+The copyright notices in the Software and this entire statement, including
+the above license grant, this restriction and the following disclaimer,
+must be included in all copies of the Software, in whole or in part, and
+all derivative works of the Software, unless such copies or derivative
+works are solely in the form of machine-executable object code generated by
+a source language processor.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT
+SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE
+FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE,
+ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
+DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+This project includes code from Apache Kudu.
+
+ * cpp/cmake_modules/CompilerInfo.cmake is based on Kudu's cmake_modules/CompilerInfo.cmake
+
+Copyright: 2016 The Apache Software Foundation.
+Home page: https://kudu.apache.org/
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This project includes code from Apache Impala (incubating), formerly
+Impala. The Impala code and rights were donated to the ASF as part of the
+Incubator process after the initial code imports into Apache Parquet.
+
+Copyright: 2012 Cloudera, Inc.
+Copyright: 2016 The Apache Software Foundation.
+Home page: http://impala.apache.org/
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This project includes code from Apache Aurora.
+
+* dev/release/{release,changelog,release-candidate} are based on the scripts from
+  Apache Aurora
+
+Copyright: 2016 The Apache Software Foundation.
+Home page: https://aurora.apache.org/
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This project includes code from the Google styleguide.
+
+* cpp/build-support/cpplint.py is based on the scripts from the Google styleguide.
+
+Copyright: 2009 Google Inc. All rights reserved.
+Homepage: https://github.com/google/styleguide
+License: 3-clause BSD
+
+--------------------------------------------------------------------------------
+
+This project includes code from Snappy.
+
+* cpp/cmake_modules/{SnappyCMakeLists.txt,SnappyConfig.h} are based on code
+  from Google's Snappy project.
+
+Copyright: 2009 Google Inc. All rights reserved.
+Homepage: https://github.com/google/snappy
+License: 3-clause BSD
+
+--------------------------------------------------------------------------------
+
+This project includes code from the manylinux project.
+
+* python/manylinux1/scripts/{build_python.sh,python-tag-abi-tag.py,
+  requirements.txt} are based on code from the manylinux project.
+
+Copyright: 2016 manylinux
+Homepage: https://github.com/pypa/manylinux
+License: The MIT License (MIT)
+
+--------------------------------------------------------------------------------
+
+This project includes code from the cymove project:
+
+* python/pyarrow/includes/common.pxd includes code from the cymove project
+
+The MIT License (MIT)
+Copyright (c) 2019 Omer Ozarslan
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM,
+DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
+OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE
+OR OTHER DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+The projects includes code from the Ursabot project under the dev/archery
+directory.
+
+License: BSD 2-Clause
+
+Copyright 2019 RStudio, Inc.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+1. Redistributions of source code must retain the above copyright notice, this
+   list of conditions and the following disclaimer.
+
+2. Redistributions in binary form must reproduce the above copyright notice,
+   this list of conditions and the following disclaimer in the documentation
+   and/or other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+This project include code from mingw-w64.
+
+* cpp/src/arrow/util/cpu-info.cc has a polyfill for mingw-w64 < 5
+
+Copyright (c) 2009 - 2013 by the mingw-w64 project
+Homepage: https://mingw-w64.org
+License: Zope Public License (ZPL) Version 2.1.
+
+---------------------------------------------------------------------------------
+
+This project include code from Google's Asylo project.
+
+* cpp/src/arrow/result.h is based on status_or.h
+
+Copyright (c)  Copyright 2017 Asylo authors
+Homepage: https://asylo.dev/
+License: Apache 2.0
+
+--------------------------------------------------------------------------------
+
+This project includes code from Google's protobuf project
+
+* cpp/src/arrow/result.h ARROW_ASSIGN_OR_RAISE is based off ASSIGN_OR_RETURN
+* cpp/src/arrow/util/bit_stream_utils.h contains code from wire_format_lite.h
+
+Copyright 2008 Google Inc.  All rights reserved.
+Homepage: https://developers.google.com/protocol-buffers/
+License:
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+    * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+Code generated by the Protocol Buffer compiler is owned by the owner
+of the input file used when generating it.  This code is not
+standalone and requires a support library to be linked with it.  This
+support library is itself covered by the above license.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency LLVM is statically linked in certain binary distributions.
+Additionally some sections of source code have been derived from sources in LLVM
+and have been clearly labeled as such. LLVM has the following license:
+
+==============================================================================
+The LLVM Project is under the Apache License v2.0 with LLVM Exceptions:
+==============================================================================
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+    TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+    1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+    2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+    3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+    4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+    5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+    6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+    7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+    8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+    9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+    END OF TERMS AND CONDITIONS
+
+    APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+    Copyright [yyyy] [name of copyright owner]
+
+    Licensed under the Apache License, Version 2.0 (the "License");
+    you may not use this file except in compliance with the License.
+    You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+
+
+---- LLVM Exceptions to the Apache 2.0 License ----
+
+As an exception, if, as a result of your compiling your source code, portions
+of this Software are embedded into an Object form of such source code, you
+may redistribute such embedded portions in such Object form without complying
+with the conditions of Sections 4(a), 4(b) and 4(d) of the License.
+
+In addition, if you combine or link compiled forms of this Software with
+software that is licensed under the GPLv2 ("Combined Software") and if a
+court of competent jurisdiction determines that the patent provision (Section
+3), the indemnity provision (Section 9) or other Section of the License
+conflicts with the conditions of the GPLv2, you may retroactively and
+prospectively choose to deem waived or otherwise exclude such Section(s) of
+the License, but only in their entirety and only with respect to the Combined
+Software.
+
+==============================================================================
+Software from third parties included in the LLVM Project:
+==============================================================================
+The LLVM Project contains third party software which is under different license
+terms. All such code will be identified clearly using at least one of two
+mechanisms:
+1) It will be in a separate directory tree with its own `LICENSE.txt` or
+   `LICENSE` file at the top containing the specific license and restrictions
+   which apply to that software, or
+2) It will contain specific license and restriction terms at the top of every
+   file.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency gRPC is statically linked in certain binary
+distributions, like the python wheels. gRPC has the following license:
+
+Copyright 2014 gRPC authors.
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency Apache Thrift is statically linked in certain binary
+distributions, like the python wheels. Apache Thrift has the following license:
+
+Apache Thrift
+Copyright (C) 2006 - 2019, The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency Apache ORC is statically linked in certain binary
+distributions, like the python wheels. Apache ORC has the following license:
+
+Apache ORC
+Copyright 2013-2019 The Apache Software Foundation
+
+This product includes software developed by The Apache Software
+Foundation (http://www.apache.org/).
+
+This product includes software developed by Hewlett-Packard:
+(c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency zstd is statically linked in certain binary
+distributions, like the python wheels. ZSTD has the following license:
+
+BSD License
+
+For Zstandard software
+
+Copyright (c) 2016-present, Facebook, Inc. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+ * Redistributions of source code must retain the above copyright notice, this
+   list of conditions and the following disclaimer.
+
+ * Redistributions in binary form must reproduce the above copyright notice,
+   this list of conditions and the following disclaimer in the documentation
+   and/or other materials provided with the distribution.
+
+ * Neither the name Facebook nor the names of its contributors may be used to
+   endorse or promote products derived from this software without specific
+   prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency lz4 is statically linked in certain binary
+distributions, like the python wheels. lz4 has the following license:
+
+LZ4 Library
+Copyright (c) 2011-2016, Yann Collet
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice, this
+  list of conditions and the following disclaimer in the documentation and/or
+  other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency Brotli is statically linked in certain binary
+distributions, like the python wheels. Brotli has the following license:
+
+Copyright (c) 2009, 2010, 2013-2016 by the Brotli Authors.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.  IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency rapidjson is statically linked in certain binary
+distributions, like the python wheels. rapidjson and its dependencies have the
+following licenses:
+
+Tencent is pleased to support the open source community by making RapidJSON
+available.
+
+Copyright (C) 2015 THL A29 Limited, a Tencent company, and Milo Yip.
+All rights reserved.
+
+If you have downloaded a copy of the RapidJSON binary from Tencent, please note
+that the RapidJSON binary is licensed under the MIT License.
+If you have downloaded a copy of the RapidJSON source code from Tencent, please
+note that RapidJSON source code is licensed under the MIT License, except for
+the third-party components listed below which are subject to different license
+terms.  Your integration of RapidJSON into your own projects may require
+compliance with the MIT License, as well as the other licenses applicable to
+the third-party components included within RapidJSON. To avoid the problematic
+JSON license in your own projects, it's sufficient to exclude the
+bin/jsonchecker/ directory, as it's the only code under the JSON license.
+A copy of the MIT License is included in this file.
+
+Other dependencies and licenses:
+
+    Open Source Software Licensed Under the BSD License:
+    --------------------------------------------------------------------
+
+    The msinttypes r29
+    Copyright (c) 2006-2013 Alexander Chemeris
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or without
+    modification, are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright notice,
+    this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright notice,
+    this list of conditions and the following disclaimer in the documentation
+    and/or other materials provided with the distribution.
+    * Neither the name of  copyright holder nor the names of its contributors
+    may be used to endorse or promote products derived from this software
+    without specific prior written permission.
+
+    THIS SOFTWARE IS PROVIDED BY THE REGENTS AND CONTRIBUTORS ``AS IS'' AND ANY
+    EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+    WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+    DISCLAIMED. IN NO EVENT SHALL THE REGENTS AND CONTRIBUTORS BE LIABLE FOR
+    ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+    DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+    SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+    CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+    LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY
+    OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH
+    DAMAGE.
+
+    Terms of the MIT License:
+    --------------------------------------------------------------------
+
+    Permission is hereby granted, free of charge, to any person obtaining a
+    copy of this software and associated documentation files (the "Software"),
+    to deal in the Software without restriction, including without limitation
+    the rights to use, copy, modify, merge, publish, distribute, sublicense,
+    and/or sell copies of the Software, and to permit persons to whom the
+    Software is furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included
+    in all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+    FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
+    DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency snappy is statically linked in certain binary
+distributions, like the python wheels. snappy has the following license:
+
+Copyright 2011, Google Inc.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright notice,
+      this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright notice,
+      this list of conditions and the following disclaimer in the documentation
+      and/or other materials provided with the distribution.
+    * Neither the name of Google Inc. nor the names of its contributors may be
+      used to endorse or promote products derived from this software without
+      specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+===
+
+Some of the benchmark data in testdata/ is licensed differently:
+
+ - fireworks.jpeg is Copyright 2013 Steinar H. Gunderson, and
+   is licensed under the Creative Commons Attribution 3.0 license
+   (CC-BY-3.0). See https://creativecommons.org/licenses/by/3.0/
+   for more information.
+
+ - kppkn.gtb is taken from the Gaviota chess tablebase set, and
+   is licensed under the MIT License. See
+   https://sites.google.com/site/gaviotachessengine/Home/endgame-tablebases-1
+   for more information.
+
+ - paper-100k.pdf is an excerpt (bytes 92160 to 194560) from the paper
+   “Combinatorial Modeling of Chromatin Features Quantitatively Predicts DNA
+   Replication Timing in _Drosophila_” by Federico Comoglio and Renato Paro,
+   which is licensed under the CC-BY license. See
+   http://www.ploscompbiol.org/static/license for more ifnormation.
+
+ - alice29.txt, asyoulik.txt, plrabn12.txt and lcet10.txt are from Project
+   Gutenberg. The first three have expired copyrights and are in the public
+   domain; the latter does not have expired copyright, but is still in the
+   public domain according to the license information
+   (http://www.gutenberg.org/ebooks/53).
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency gflags is statically linked in certain binary
+distributions, like the python wheels. gflags has the following license:
+
+Copyright (c) 2006, Google Inc.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+    * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency glog is statically linked in certain binary
+distributions, like the python wheels. glog has the following license:
+
+Copyright (c) 2008, Google Inc.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+    * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+A function gettimeofday in utilities.cc is based on
+
+http://www.google.com/codesearch/p?hl=en#dR3YEbitojA/COPYING&q=GetSystemTimeAsFileTime%20license:bsd
+
+The license of this code is:
+
+Copyright (c) 2003-2008, Jouni Malinen <j@w1.fi> and contributors
+All Rights Reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+1. Redistributions of source code must retain the above copyright
+   notice, this list of conditions and the following disclaimer.
+
+2. Redistributions in binary form must reproduce the above copyright
+   notice, this list of conditions and the following disclaimer in the
+   documentation and/or other materials provided with the distribution.
+
+3. Neither the name(s) of the above-listed copyright holder(s) nor the
+   names of its contributors may be used to endorse or promote products
+   derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency re2 is statically linked in certain binary
+distributions, like the python wheels. re2 has the following license:
+
+Copyright (c) 2009 The RE2 Authors. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+      copyright notice, this list of conditions and the following
+      disclaimer in the documentation and/or other materials provided
+      with the distribution.
+    * Neither the name of Google Inc. nor the names of its contributors
+      may be used to endorse or promote products derived from this
+      software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency c-ares is statically linked in certain binary
+distributions, like the python wheels. c-ares has the following license:
+
+# c-ares license
+
+Copyright (c) 2007 - 2018, Daniel Stenberg with many contributors, see AUTHORS
+file.
+
+Copyright 1998 by the Massachusetts Institute of Technology.
+
+Permission to use, copy, modify, and distribute this software and its
+documentation for any purpose and without fee is hereby granted, provided that
+the above copyright notice appear in all copies and that both that copyright
+notice and this permission notice appear in supporting documentation, and that
+the name of M.I.T. not be used in advertising or publicity pertaining to
+distribution of the software without specific, written prior permission.
+M.I.T. makes no representations about the suitability of this software for any
+purpose.  It is provided "as is" without express or implied warranty.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency zlib is redistributed as a dynamically linked shared
+library in certain binary distributions, like the python wheels. In the future
+this will likely change to static linkage. zlib has the following license:
+
+zlib.h -- interface of the 'zlib' general purpose compression library
+  version 1.2.11, January 15th, 2017
+
+  Copyright (C) 1995-2017 Jean-loup Gailly and Mark Adler
+
+  This software is provided 'as-is', without any express or implied
+  warranty.  In no event will the authors be held liable for any damages
+  arising from the use of this software.
+
+  Permission is granted to anyone to use this software for any purpose,
+  including commercial applications, and to alter it and redistribute it
+  freely, subject to the following restrictions:
+
+  1. The origin of this software must not be misrepresented; you must not
+     claim that you wrote the original software. If you use this software
+     in a product, an acknowledgment in the product documentation would be
+     appreciated but is not required.
+  2. Altered source versions must be plainly marked as such, and must not be
+     misrepresented as being the original software.
+  3. This notice may not be removed or altered from any source distribution.
+
+  Jean-loup Gailly        Mark Adler
+  jloup@gzip.org          madler@alumni.caltech.edu
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency openssl is redistributed as a dynamically linked shared
+library in certain binary distributions, like the python wheels. openssl
+preceding version 3 has the following license:
+
+  LICENSE ISSUES
+  ==============
+
+  The OpenSSL toolkit stays under a double license, i.e. both the conditions of
+  the OpenSSL License and the original SSLeay license apply to the toolkit.
+  See below for the actual license texts.
+
+  OpenSSL License
+  ---------------
+
+/* ====================================================================
+ * Copyright (c) 1998-2019 The OpenSSL Project.  All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ *
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ *
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in
+ *    the documentation and/or other materials provided with the
+ *    distribution.
+ *
+ * 3. All advertising materials mentioning features or use of this
+ *    software must display the following acknowledgment:
+ *    "This product includes software developed by the OpenSSL Project
+ *    for use in the OpenSSL Toolkit. (http://www.openssl.org/)"
+ *
+ * 4. The names "OpenSSL Toolkit" and "OpenSSL Project" must not be used to
+ *    endorse or promote products derived from this software without
+ *    prior written permission. For written permission, please contact
+ *    openssl-core@openssl.org.
+ *
+ * 5. Products derived from this software may not be called "OpenSSL"
+ *    nor may "OpenSSL" appear in their names without prior written
+ *    permission of the OpenSSL Project.
+ *
+ * 6. Redistributions of any form whatsoever must retain the following
+ *    acknowledgment:
+ *    "This product includes software developed by the OpenSSL Project
+ *    for use in the OpenSSL Toolkit (http://www.openssl.org/)"
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE OpenSSL PROJECT ``AS IS'' AND ANY
+ * EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
+ * PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL THE OpenSSL PROJECT OR
+ * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
+ * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+ * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
+ * OF THE POSSIBILITY OF SUCH DAMAGE.
+ * ====================================================================
+ *
+ * This product includes cryptographic software written by Eric Young
+ * (eay@cryptsoft.com).  This product includes software written by Tim
+ * Hudson (tjh@cryptsoft.com).
+ *
+ */
+
+ Original SSLeay License
+ -----------------------
+
+/* Copyright (C) 1995-1998 Eric Young (eay@cryptsoft.com)
+ * All rights reserved.
+ *
+ * This package is an SSL implementation written
+ * by Eric Young (eay@cryptsoft.com).
+ * The implementation was written so as to conform with Netscapes SSL.
+ *
+ * This library is free for commercial and non-commercial use as long as
+ * the following conditions are aheared to.  The following conditions
+ * apply to all code found in this distribution, be it the RC4, RSA,
+ * lhash, DES, etc., code; not just the SSL code.  The SSL documentation
+ * included with this distribution is covered by the same copyright terms
+ * except that the holder is Tim Hudson (tjh@cryptsoft.com).
+ *
+ * Copyright remains Eric Young's, and as such any Copyright notices in
+ * the code are not to be removed.
+ * If this package is used in a product, Eric Young should be given attribution
+ * as the author of the parts of the library used.
+ * This can be in the form of a textual message at program startup or
+ * in documentation (online or textual) provided with the package.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ * 3. All advertising materials mentioning features or use of this software
+ *    must display the following acknowledgement:
+ *    "This product includes cryptographic software written by
+ *     Eric Young (eay@cryptsoft.com)"
+ *    The word 'cryptographic' can be left out if the rouines from the library
+ *    being used are not cryptographic related :-).
+ * 4. If you include any Windows specific code (or a derivative thereof) from
+ *    the apps directory (application code) you must include an acknowledgement:
+ *    "This product includes software written by Tim Hudson (tjh@cryptsoft.com)"
+ *
+ * THIS SOFTWARE IS PROVIDED BY ERIC YOUNG ``AS IS'' AND
+ * ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ * ARE DISCLAIMED.  IN NO EVENT SHALL THE AUTHOR OR CONTRIBUTORS BE LIABLE
+ * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+ * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS
+ * OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
+ * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY
+ * OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
+ * SUCH DAMAGE.
+ *
+ * The licence and distribution terms for any publically available version or
+ * derivative of this code cannot be changed.  i.e. this code cannot simply be
+ * copied and put under another distribution licence
+ * [including the GNU Public Licence.]
+ */
+
+--------------------------------------------------------------------------------
+
+This project includes code from the rtools-backports project.
+
+* ci/scripts/PKGBUILD and ci/scripts/r_windows_build.sh are based on code
+  from the rtools-backports project.
+
+Copyright: Copyright (c) 2013 - 2019, Алексей and Jeroen Ooms.
+All rights reserved.
+Homepage: https://github.com/r-windows/rtools-backports
+License: 3-clause BSD
+
+--------------------------------------------------------------------------------
+
+Some code from pandas has been adapted for the pyarrow codebase. pandas is
+available under the 3-clause BSD license, which follows:
+
+pandas license
+==============
+
+Copyright (c) 2011-2012, Lambda Foundry, Inc. and PyData Development Team
+All rights reserved.
+
+Copyright (c) 2008-2011 AQR Capital Management, LLC
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+       notice, this list of conditions and the following disclaimer.
+
+    * Redistributions in binary form must reproduce the above
+       copyright notice, this list of conditions and the following
+       disclaimer in the documentation and/or other materials provided
+       with the distribution.
+
+    * Neither the name of the copyright holder nor the names of any
+       contributors may be used to endorse or promote products derived
+       from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDER AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+Some bits from DyND, in particular aspects of the build system, have been
+adapted from libdynd and dynd-python under the terms of the BSD 2-clause
+license
+
+The BSD 2-Clause License
+
+    Copyright (C) 2011-12, Dynamic NDArray Developers
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or without
+    modification, are permitted provided that the following conditions are
+    met:
+
+        * Redistributions of source code must retain the above copyright
+           notice, this list of conditions and the following disclaimer.
+
+        * Redistributions in binary form must reproduce the above
+           copyright notice, this list of conditions and the following
+           disclaimer in the documentation and/or other materials provided
+           with the distribution.
+
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+    "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+    LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+    A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+    OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+    SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+    LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+    DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+    THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+    (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+    OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+Dynamic NDArray Developers list:
+
+ * Mark Wiebe
+ * Continuum Analytics
+
+--------------------------------------------------------------------------------
+
+Some source code from Ibis (https://github.com/cloudera/ibis) has been adapted
+for PyArrow. Ibis is released under the Apache License, Version 2.0.
+
+--------------------------------------------------------------------------------
+
+dev/tasks/homebrew-formulae/apache-arrow.rb has the following license:
+
+BSD 2-Clause License
+
+Copyright (c) 2009-present, Homebrew contributors
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+----------------------------------------------------------------------
+
+cpp/src/arrow/vendored/base64.cpp has the following license
+
+ZLIB License
+
+Copyright (C) 2004-2017 René Nyffenegger
+
+This source code is provided 'as-is', without any express or implied
+warranty. In no event will the author be held liable for any damages arising
+from the use of this software.
+
+Permission is granted to anyone to use this software for any purpose, including
+commercial applications, and to alter it and redistribute it freely, subject to
+the following restrictions:
+
+1. The origin of this source code must not be misrepresented; you must not
+   claim that you wrote the original source code. If you use this source code
+   in a product, an acknowledgment in the product documentation would be
+   appreciated but is not required.
+
+2. Altered source versions must be plainly marked as such, and must not be
+   misrepresented as being the original source code.
+
+3. This notice may not be removed or altered from any source distribution.
+
+René Nyffenegger rene.nyffenegger@adp-gmbh.ch
+
+--------------------------------------------------------------------------------
+
+This project includes code from Folly.
+
+ * cpp/src/arrow/vendored/ProducerConsumerQueue.h
+
+is based on Folly's
+
+ * folly/Portability.h
+ * folly/lang/Align.h
+ * folly/ProducerConsumerQueue.h
+
+Copyright: Copyright (c) Facebook, Inc. and its affiliates.
+Home page: https://github.com/facebook/folly
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+The file cpp/src/arrow/vendored/musl/strptime.c has the following license
+
+Copyright © 2005-2020 Rich Felker, et al.
+
+Permission is hereby granted, free of charge, to any person obtaining
+a copy of this software and associated documentation files (the
+"Software"), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to
+the following conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY
+CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE
+SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+The file cpp/cmake_modules/BuildUtils.cmake contains code from
+
+https://gist.github.com/cristianadam/ef920342939a89fae3e8a85ca9459b49
+
+which is made available under the MIT license
+
+Copyright (c) 2019 Cristian Adam
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+The files in cpp/src/arrow/vendored/portable-snippets/ contain code from
+
+https://github.com/nemequ/portable-snippets
+
+and have the following copyright notice:
+
+Each source file contains a preamble explaining the license situation
+for that file, which takes priority over this file.  With the
+exception of some code pulled in from other repositories (such as
+µnit, an MIT-licensed project which is used for testing), the code is
+public domain, released using the CC0 1.0 Universal dedication (*).
+
+(*) https://creativecommons.org/publicdomain/zero/1.0/legalcode
+
+--------------------------------------------------------------------------------
+
+The files in cpp/src/arrow/vendored/fast_float/ contain code from
+
+https://github.com/lemire/fast_float
+
+which is made available under the Apache License 2.0.
+
+--------------------------------------------------------------------------------
+
+The file python/pyarrow/vendored/docscrape.py contains code from
+
+https://github.com/numpy/numpydoc/
+
+which is made available under the BSD 2-clause license.
+
+--------------------------------------------------------------------------------
+
+The file python/pyarrow/vendored/version.py contains code from
+
+https://github.com/pypa/packaging/
+
+which is made available under both the Apache license v2.0 and the
+BSD 2-clause license.
+
+--------------------------------------------------------------------------------
+
+The files in cpp/src/arrow/vendored/pcg contain code from
+
+https://github.com/imneme/pcg-cpp
+
+and have the following copyright notice:
+
+Copyright 2014-2019 Melissa O'Neill <oneill@pcg-random.org>,
+                    and the PCG Project contributors.
+
+SPDX-License-Identifier: (Apache-2.0 OR MIT)
+
+Licensed under the Apache License, Version 2.0 (provided in
+LICENSE-APACHE.txt and at http://www.apache.org/licenses/LICENSE-2.0)
+or under the MIT license (provided in LICENSE-MIT.txt and at
+http://opensource.org/licenses/MIT), at your option. This file may not
+be copied, modified, or distributed except according to those terms.
+
+Distributed on an "AS IS" BASIS, WITHOUT WARRANTY OF ANY KIND, either
+express or implied.  See your chosen license for details.
+
+--------------------------------------------------------------------------------
+r/R/dplyr-count-tally.R (some portions)
+
+Some portions of this file are derived from code from
+
+https://github.com/tidyverse/dplyr/
+
+which is made available under the MIT license
+
+Copyright (c) 2013-2019 RStudio and others.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the “Software”), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED “AS IS”, WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+The file src/arrow/util/io_util.cc contains code from the CPython project
+which is made available under the Python Software Foundation License Version 2.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency opentelemetry-cpp is statically linked in certain binary
+distributions. opentelemetry-cpp is made available under the Apache License 2.0.
+
+Copyright The OpenTelemetry Authors
+SPDX-License-Identifier: Apache-2.0
+
+--------------------------------------------------------------------------------
+
+ci/conan/ is based on code from Conan Package and Dependency Manager.
+
+Copyright (c) 2019 Conan.io
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+3rdparty dependency UCX is redistributed as a dynamically linked shared
+library in certain binary distributions. UCX has the following license:
+
+Copyright (c) 2014-2015      UT-Battelle, LLC. All rights reserved.
+Copyright (C) 2014-2020      Mellanox Technologies Ltd. All rights reserved.
+Copyright (C) 2014-2015      The University of Houston System. All rights reserved.
+Copyright (C) 2015           The University of Tennessee and The University
+                             of Tennessee Research Foundation. All rights reserved.
+Copyright (C) 2016-2020      ARM Ltd. All rights reserved.
+Copyright (c) 2016           Los Alamos National Security, LLC. All rights reserved.
+Copyright (C) 2016-2020      Advanced Micro Devices, Inc.  All rights reserved.
+Copyright (C) 2019           UChicago Argonne, LLC.  All rights reserved.
+Copyright (c) 2018-2020      NVIDIA CORPORATION. All rights reserved.
+Copyright (C) 2020           Huawei Technologies Co., Ltd. All rights reserved.
+Copyright (C) 2016-2020      Stony Brook University. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions
+are met:
+
+1. Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+2. Redistributions in binary form must reproduce the above copyright
+notice, this list of conditions and the following disclaimer in the
+documentation and/or other materials provided with the distribution.
+3. Neither the name of the copyright holder nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED
+TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+The file dev/tasks/r/github.packages.yml contains code from
+
+https://github.com/ursa-labs/arrow-r-nightly
+
+which is made available under the Apache License 2.0.
+
+--------------------------------------------------------------------------------
+.github/actions/sync-nightlies/action.yml  (some portions)
+
+Some portions of this file are derived from code from
+
+https://github.com/JoshPiper/rsync-docker
+
+which is made available under the MIT license
+
+Copyright (c) 2020 Joshua Piper
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+--------------------------------------------------------------------------------
+.github/actions/sync-nightlies/action.yml (some portions)
+
+Some portions of this file are derived from code from
+
+https://github.com/burnett01/rsync-deployments
+
+which is made available under the MIT license
+
+Copyright (c) 2019-2022 Contention
+Copyright (c) 2019-2022 Burnett01
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+--------------------------------------------------------------------------------
+java/vector/src/main/java/org/apache/arrow/vector/util/IntObjectHashMap.java
+java/vector/src/main/java/org/apache/arrow/vector/util/IntObjectMap.java
+
+These file are derived from code from Netty, which is made available under the
+Apache License 2.0.

+ 84 - 0
x-pack/plugin/esql/arrow/licenses/arrow-NOTICE.txt

@@ -0,0 +1,84 @@
+Apache Arrow
+Copyright 2016-2024 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+This product includes software from the SFrame project (BSD, 3-clause).
+* Copyright (C) 2015 Dato, Inc.
+* Copyright (c) 2009 Carnegie Mellon University.
+
+This product includes software from the Feather project (Apache 2.0)
+https://github.com/wesm/feather
+
+This product includes software from the DyND project (BSD 2-clause)
+https://github.com/libdynd
+
+This product includes software from the LLVM project
+ * distributed under the University of Illinois Open Source
+
+This product includes software from the google-lint project
+ * Copyright (c) 2009 Google Inc. All rights reserved.
+
+This product includes software from the mman-win32 project
+ * Copyright https://code.google.com/p/mman-win32/
+ * Licensed under the MIT License;
+
+This product includes software from the LevelDB project
+ * Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+ * Use of this source code is governed by a BSD-style license that can be
+ * Moved from Kudu http://github.com/cloudera/kudu
+
+This product includes software from the CMake project
+ * Copyright 2001-2009 Kitware, Inc.
+ * Copyright 2012-2014 Continuum Analytics, Inc.
+ * All rights reserved.
+
+This product includes software from https://github.com/matthew-brett/multibuild (BSD 2-clause)
+ * Copyright (c) 2013-2016, Matt Terry and Matthew Brett; all rights reserved.
+
+This product includes software from the Ibis project (Apache 2.0)
+ * Copyright (c) 2015 Cloudera, Inc.
+ * https://github.com/cloudera/ibis
+
+This product includes software from Dremio (Apache 2.0)
+  * Copyright (C) 2017-2018 Dremio Corporation
+  * https://github.com/dremio/dremio-oss
+
+This product includes software from Google Guava (Apache 2.0)
+  * Copyright (C) 2007 The Guava Authors
+  * https://github.com/google/guava
+
+This product include software from CMake (BSD 3-Clause)
+  * CMake - Cross Platform Makefile Generator
+  * Copyright 2000-2019 Kitware, Inc. and Contributors
+
+The web site includes files generated by Jekyll.
+
+--------------------------------------------------------------------------------
+
+This product includes code from Apache Kudu, which includes the following in
+its NOTICE file:
+
+  Apache Kudu
+  Copyright 2016 The Apache Software Foundation
+
+  This product includes software developed at
+  The Apache Software Foundation (http://www.apache.org/).
+
+  Portions of this software were developed at
+  Cloudera, Inc (http://www.cloudera.com/).
+
+--------------------------------------------------------------------------------
+
+This product includes code from Apache ORC, which includes the following in
+its NOTICE file:
+
+  Apache ORC
+  Copyright 2013-2019 The Apache Software Foundation
+
+  This product includes software developed by The Apache Software
+  Foundation (http://www.apache.org/).
+
+  This product includes software developed by Hewlett-Packard:
+  (c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P

+ 22 - 0
x-pack/plugin/esql/arrow/licenses/checker-qual-LICENSE.txt

@@ -0,0 +1,22 @@
+Checker Framework qualifiers
+Copyright 2004-present by the Checker Framework developers
+
+MIT License:
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.

+ 0 - 0
x-pack/plugin/esql/arrow/licenses/checker-qual-NOTICE.txt


+ 202 - 0
x-pack/plugin/esql/arrow/licenses/flatbuffers-java-LICENSE.txt

@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.

+ 0 - 0
x-pack/plugin/esql/arrow/licenses/flatbuffers-java-NOTICE.txt


+ 202 - 0
x-pack/plugin/esql/arrow/licenses/jackson-LICENSE.txt

@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.

+ 0 - 0
x-pack/plugin/esql/arrow/licenses/jackson-NOTICE.txt


+ 21 - 0
x-pack/plugin/esql/arrow/licenses/slf4j-LICENSE.txt

@@ -0,0 +1,21 @@
+Copyright (c) 2004-2022 QOS.ch Sarl (Switzerland)
+All rights reserved.
+
+Permission is hereby granted, free  of charge, to any person obtaining
+a  copy  of this  software  and  associated  documentation files  (the
+"Software"), to  deal in  the Software without  restriction, including
+without limitation  the rights to  use, copy, modify,  merge, publish,
+distribute,  sublicense, and/or sell  copies of  the Software,  and to
+permit persons to whom the Software  is furnished to do so, subject to
+the following conditions:
+
+The  above  copyright  notice  and  this permission  notice  shall  be
+included in all copies or substantial portions of the Software.
+
+THE  SOFTWARE IS  PROVIDED  "AS  IS", WITHOUT  WARRANTY  OF ANY  KIND,
+EXPRESS OR  IMPLIED, INCLUDING  BUT NOT LIMITED  TO THE  WARRANTIES OF
+MERCHANTABILITY,    FITNESS    FOR    A   PARTICULAR    PURPOSE    AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+OF CONTRACT, TORT OR OTHERWISE,  ARISING FROM, OUT OF OR IN CONNECTION
+WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.

+ 0 - 0
x-pack/plugin/esql/arrow/licenses/slf4j-NOTICE.txt


+ 69 - 0
x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/AllocationManagerShim.java

@@ -0,0 +1,69 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.arrow;
+
+import org.apache.arrow.memory.AllocationManager;
+import org.apache.arrow.memory.ArrowBuf;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.DefaultAllocationManagerOption;
+import org.elasticsearch.core.SuppressForbidden;
+import org.elasticsearch.logging.LogManager;
+import org.elasticsearch.logging.Logger;
+
+import java.lang.reflect.Field;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+
+/**
+ * An Arrow memory allocation manager that always fails.
+ * <p>
+ * We don't actually use Arrow's memory manager as we stream dataframe buffers directly from ESQL blocks.
+ * But Arrow won't initialize properly unless it has one (and requires either the arrow-memory-netty or arrow-memory-unsafe libraries).
+ * It also does some fancy classpath scanning and calls to {@code setAccessible} which will be rejected by the security manager.
+ * <p>
+ * So we configure an allocation manager that will fail on any attempt to allocate memory.
+ *
+ * @see DefaultAllocationManagerOption
+ */
+public class AllocationManagerShim implements AllocationManager.Factory {
+
+    private static final Logger logger = LogManager.getLogger(AllocationManagerShim.class);
+
+    /**
+     * Initialize the Arrow memory allocation manager shim.
+     */
+    @SuppressForbidden(reason = "Inject the default Arrow memory allocation manager")
+    public static void init() {
+        try {
+            Class.forName("org.elasticsearch.test.ESTestCase");
+            logger.info("We're in tests, not disabling Arrow memory manager so we can use a real runtime for testing");
+        } catch (ClassNotFoundException notfound) {
+            logger.debug("Disabling Arrow's allocation manager");
+            AccessController.doPrivileged((PrivilegedAction<Void>) () -> {
+                try {
+                    Field field = DefaultAllocationManagerOption.class.getDeclaredField("DEFAULT_ALLOCATION_MANAGER_FACTORY");
+                    field.setAccessible(true);
+                    field.set(null, new AllocationManagerShim());
+                } catch (Exception e) {
+                    throw new AssertionError("Can't init Arrow", e);
+                }
+                return null;
+            });
+        }
+    }
+
+    @Override
+    public AllocationManager create(BufferAllocator accountingAllocator, long size) {
+        throw new UnsupportedOperationException("Arrow memory manager is disabled");
+    }
+
+    @Override
+    public ArrowBuf empty() {
+        throw new UnsupportedOperationException("Arrow memory manager is disabled");
+    }
+}

+ 35 - 0
x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/ArrowFormat.java

@@ -0,0 +1,35 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.arrow;
+
+import org.elasticsearch.xcontent.MediaType;
+
+import java.util.Map;
+import java.util.Set;
+
+public class ArrowFormat implements MediaType {
+    public static final ArrowFormat INSTANCE = new ArrowFormat();
+
+    private static final String FORMAT = "arrow";
+    // See https://www.iana.org/assignments/media-types/application/vnd.apache.arrow.stream
+    public static final String CONTENT_TYPE = "application/vnd.apache.arrow.stream";
+    private static final String VENDOR_CONTENT_TYPE = "application/vnd.elasticsearch+arrow+stream";
+
+    @Override
+    public String queryParameter() {
+        return FORMAT;
+    }
+
+    @Override
+    public Set<HeaderValue> headerValues() {
+        return Set.of(
+            new HeaderValue(CONTENT_TYPE, Map.of("header", "present|absent")),
+            new HeaderValue(VENDOR_CONTENT_TYPE, Map.of("header", "present|absent", COMPATIBLE_WITH_PARAMETER_NAME, VERSION_PATTERN))
+        );
+    }
+}

+ 379 - 0
x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/ArrowResponse.java

@@ -0,0 +1,379 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.arrow;
+
+import org.apache.arrow.memory.ArrowBuf;
+import org.apache.arrow.vector.compression.NoCompressionCodec;
+import org.apache.arrow.vector.ipc.ArrowStreamWriter;
+import org.apache.arrow.vector.ipc.WriteChannel;
+import org.apache.arrow.vector.ipc.message.ArrowFieldNode;
+import org.apache.arrow.vector.ipc.message.ArrowRecordBatch;
+import org.apache.arrow.vector.ipc.message.IpcOption;
+import org.apache.arrow.vector.ipc.message.MessageSerializer;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.common.bytes.BytesReference;
+import org.elasticsearch.common.bytes.ReleasableBytesReference;
+import org.elasticsearch.common.io.stream.BytesStream;
+import org.elasticsearch.common.io.stream.RecyclerBytesStreamOutput;
+import org.elasticsearch.common.recycler.Recycler;
+import org.elasticsearch.compute.data.Block;
+import org.elasticsearch.compute.data.Page;
+import org.elasticsearch.core.Releasable;
+import org.elasticsearch.core.Releasables;
+import org.elasticsearch.rest.ChunkedRestResponseBodyPart;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+public class ArrowResponse implements ChunkedRestResponseBodyPart, Releasable {
+
+    public static class Column {
+        private final BlockConverter converter;
+        private final String name;
+
+        public Column(String esqlType, String name) {
+            this.converter = ESQL_CONVERTERS.get(esqlType);
+            if (converter == null) {
+                throw new IllegalArgumentException("ES|QL type [" + esqlType + "] is not supported by the Arrow format");
+            }
+            this.name = name;
+        }
+    }
+
+    private final List<Column> columns;
+    private Iterator<ResponseSegment> segments;
+    private ResponseSegment currentSegment;
+
+    public ArrowResponse(List<Column> columns, List<Page> pages) {
+        this.columns = columns;
+
+        currentSegment = new SchemaResponse(this);
+        List<ResponseSegment> rest = new ArrayList<>(pages.size());
+        for (int p = 0; p < pages.size(); p++) {
+            var page = pages.get(p);
+            rest.add(new PageResponse(this, page));
+            // Multivalued fields are not supported yet.
+            for (int b = 0; b < page.getBlockCount(); b++) {
+                if (page.getBlock(b).mayHaveMultivaluedFields()) {
+                    throw new IllegalArgumentException(
+                        "ES|QL response field [" + columns.get(b).name + "] is multi-valued. This isn't supported yet by the Arrow format"
+                    );
+                }
+            }
+        }
+        rest.add(new EndResponse(this));
+        segments = rest.iterator();
+    }
+
+    @Override
+    public boolean isPartComplete() {
+        return currentSegment == null;
+    }
+
+    @Override
+    public boolean isLastPart() {
+        // Even if sent in chunks, the entirety of ESQL data is available, so it's single (chunked) part
+        return true;
+    }
+
+    @Override
+    public void getNextPart(ActionListener<ChunkedRestResponseBodyPart> listener) {
+        listener.onFailure(new IllegalStateException("no continuations available"));
+    }
+
+    @Override
+    public ReleasableBytesReference encodeChunk(int sizeHint, Recycler<BytesRef> recycler) throws IOException {
+        try {
+            return currentSegment.encodeChunk(sizeHint, recycler);
+        } finally {
+            if (currentSegment.isDone()) {
+                currentSegment = segments.hasNext() ? segments.next() : null;
+            }
+        }
+    }
+
+    @Override
+    public String getResponseContentTypeString() {
+        return ArrowFormat.CONTENT_TYPE;
+    }
+
+    @Override
+    public void close() {
+        currentSegment = null;
+        segments = null;
+    }
+
+    /**
+     * An Arrow response is composed of different segments, each being a set of chunks:
+     * the schema header, the data buffers, and the trailer.
+     */
+    protected abstract static class ResponseSegment {
+        static {
+            // Init the Arrow memory manager shim
+            AllocationManagerShim.init();
+        }
+
+        protected final ArrowResponse response;
+
+        ResponseSegment(ArrowResponse response) {
+            this.response = response;
+        }
+
+        public final ReleasableBytesReference encodeChunk(int sizeHint, Recycler<BytesRef> recycler) throws IOException {
+            RecyclerBytesStreamOutput output = new RecyclerBytesStreamOutput(recycler);
+            try {
+                encodeChunk(sizeHint, output);
+                BytesReference ref = output.bytes();
+                RecyclerBytesStreamOutput closeRef = output;
+                output = null;
+                ReleasableBytesReference result = new ReleasableBytesReference(ref, () -> Releasables.closeExpectNoException(closeRef));
+                return result;
+            } finally {
+                Releasables.closeExpectNoException(output);
+            }
+        }
+
+        protected abstract void encodeChunk(int sizeHint, RecyclerBytesStreamOutput out) throws IOException;
+
+        protected abstract boolean isDone();
+
+        /**
+         * Adapts a {@link BytesStream} so that Arrow can write to it.
+         */
+        protected static WritableByteChannel arrowOut(BytesStream output) {
+            return new WritableByteChannel() {
+                @Override
+                public int write(ByteBuffer byteBuffer) throws IOException {
+                    if (byteBuffer.hasArray() == false) {
+                        throw new AssertionError("only implemented for array backed buffers");
+                    }
+                    int length = byteBuffer.remaining();
+                    output.write(byteBuffer.array(), byteBuffer.arrayOffset() + byteBuffer.position(), length);
+                    byteBuffer.position(byteBuffer.position() + length);
+                    assert byteBuffer.hasRemaining() == false;
+                    return length;
+                }
+
+                @Override
+                public boolean isOpen() {
+                    return true;
+                }
+
+                @Override
+                public void close() {}
+            };
+        }
+    }
+
+    /**
+     * Header part of the Arrow response containing the dataframe schema.
+     *
+     * @see <a href="https://arrow.apache.org/docs/format/Columnar.html#ipc-streaming-format">IPC Streaming Format</a>
+     */
+    private static class SchemaResponse extends ResponseSegment {
+        private boolean done = false;
+
+        SchemaResponse(ArrowResponse response) {
+            super(response);
+        }
+
+        @Override
+        public boolean isDone() {
+            return done;
+        }
+
+        @Override
+        protected void encodeChunk(int sizeHint, RecyclerBytesStreamOutput out) throws IOException {
+            WriteChannel arrowOut = new WriteChannel(arrowOut(out));
+            MessageSerializer.serialize(arrowOut, arrowSchema());
+            done = true;
+        }
+
+        private Schema arrowSchema() {
+            return new Schema(response.columns.stream().map(c -> new Field(c.name, c.converter.arrowFieldType(), List.of())).toList());
+        }
+    }
+
+    /**
+     * Page response segment: write an ES|QL page as an Arrow RecordBatch
+     */
+    private static class PageResponse extends ResponseSegment {
+        private final Page page;
+        private boolean done = false;
+
+        PageResponse(ArrowResponse response, Page page) {
+            super(response);
+            this.page = page;
+        }
+
+        @Override
+        public boolean isDone() {
+            return done;
+        }
+
+        // Writes some data and returns the number of bytes written.
+        interface BufWriter {
+            long write() throws IOException;
+        }
+
+        @Override
+        protected void encodeChunk(int sizeHint, RecyclerBytesStreamOutput out) throws IOException {
+            // An Arrow record batch consists of:
+            // - fields metadata, giving the number of items and the number of null values for each field
+            // - data buffers for each field. The number of buffers for a field depends on its type, e.g.:
+            // - for primitive types, there's a validity buffer (for nulls) and a value buffer.
+            // - for strings, there's a validity buffer, an offsets buffer and a data buffer
+            // See https://arrow.apache.org/docs/format/Columnar.html#recordbatch-message
+
+            // Field metadata
+            List<ArrowFieldNode> nodes = new ArrayList<>(page.getBlockCount());
+
+            // Buffers added to the record batch. They're used to track data size so that Arrow can compute offsets
+            // but contain no data. Actual writing will be done by the bufWriters. This avoids having to deal with
+            // Arrow's memory management, and in the future will allow direct write from ESQL block vectors.
+            List<ArrowBuf> bufs = new ArrayList<>(page.getBlockCount() * 2);
+
+            // Closures that will actually write a Block's data. Maps 1:1 to `bufs`.
+            List<BlockConverter.BufWriter> bufWriters = new ArrayList<>(page.getBlockCount() * 2);
+
+            // Give Arrow a WriteChannel that will iterate on `bufWriters` when requested to write a buffer.
+            WriteChannel arrowOut = new WriteChannel(arrowOut(out)) {
+                int bufIdx = 0;
+                long extraPosition = 0;
+
+                @Override
+                public void write(ArrowBuf buffer) throws IOException {
+                    extraPosition += bufWriters.get(bufIdx++).write(out);
+                }
+
+                @Override
+                public long getCurrentPosition() {
+                    return super.getCurrentPosition() + extraPosition;
+                }
+
+                @Override
+                public long align() throws IOException {
+                    int trailingByteSize = (int) (getCurrentPosition() % 8);
+                    if (trailingByteSize != 0) { // align on 8 byte boundaries
+                        return writeZeros(8 - trailingByteSize);
+                    }
+                    return 0;
+                }
+            };
+
+            // Create Arrow buffers for each of the blocks in this page
+            for (int b = 0; b < page.getBlockCount(); b++) {
+                var converter = response.columns.get(b).converter;
+
+                Block block = page.getBlock(b);
+                nodes.add(new ArrowFieldNode(block.getPositionCount(), converter.nullValuesCount(block)));
+                converter.convert(block, bufs, bufWriters);
+            }
+
+            // Create the batch and serialize it
+            ArrowRecordBatch batch = new ArrowRecordBatch(
+                page.getPositionCount(),
+                nodes,
+                bufs,
+                NoCompressionCodec.DEFAULT_BODY_COMPRESSION,
+                true, // align buffers
+                false // retain buffers
+            );
+            MessageSerializer.serialize(arrowOut, batch);
+
+            done = true; // one day we should respect sizeHint here. kindness.
+        }
+    }
+
+    /**
+     * Trailer segment: write the Arrow end of stream marker
+     */
+    private static class EndResponse extends ResponseSegment {
+        private boolean done = false;
+
+        private EndResponse(ArrowResponse response) {
+            super(response);
+        }
+
+        @Override
+        public boolean isDone() {
+            return done;
+        }
+
+        @Override
+        protected void encodeChunk(int sizeHint, RecyclerBytesStreamOutput out) throws IOException {
+            ArrowStreamWriter.writeEndOfStream(new WriteChannel(arrowOut(out)), IpcOption.DEFAULT);
+            done = true;
+        }
+    }
+
+    /**
+     * Converters for every ES|QL type
+     */
+    static final Map<String, BlockConverter> ESQL_CONVERTERS = Map.ofEntries(
+        // For reference:
+        // - EsqlDataTypes: list of ESQL data types (not all are present in outputs)
+        // - PositionToXContent: conversions for ESQL JSON output
+        // - EsqlDataTypeConverter: conversions to ESQL datatypes
+        // Missing: multi-valued values
+
+        buildEntry(new BlockConverter.AsNull("null")),
+        buildEntry(new BlockConverter.AsNull("unsupported")),
+
+        buildEntry(new BlockConverter.AsBoolean("boolean")),
+
+        buildEntry(new BlockConverter.AsInt32("integer")),
+        buildEntry(new BlockConverter.AsInt32("counter_integer")),
+
+        buildEntry(new BlockConverter.AsInt64("long")),
+        // FIXME: counters: are they signed?
+        buildEntry(new BlockConverter.AsInt64("counter_long")),
+        buildEntry(new BlockConverter.AsInt64("unsigned_long", MinorType.UINT8)),
+
+        buildEntry(new BlockConverter.AsFloat64("double")),
+        buildEntry(new BlockConverter.AsFloat64("counter_double")),
+
+        buildEntry(new BlockConverter.AsVarChar("keyword")),
+        buildEntry(new BlockConverter.AsVarChar("text")),
+
+        // date: array of int64 seconds since epoch
+        // FIXME: is it signed?
+        buildEntry(new BlockConverter.AsInt64("date", MinorType.TIMESTAMPMILLI)),
+
+        // ip are represented as 16-byte ipv6 addresses. We shorten mapped ipv4 addresses to 4 bytes.
+        // Another option would be to use a fixed size binary to avoid the offset array. But with mostly
+        // ipv4 addresses it would still be twice as big.
+        buildEntry(new BlockConverter.TransformedBytesRef("ip", MinorType.VARBINARY, ValueConversions::shortenIpV4Addresses)),
+
+        // geo_point: Keep WKB format (JSON converts to WKT)
+        buildEntry(new BlockConverter.AsVarBinary("geo_point")),
+        buildEntry(new BlockConverter.AsVarBinary("geo_shape")),
+        buildEntry(new BlockConverter.AsVarBinary("cartesian_point")),
+        buildEntry(new BlockConverter.AsVarBinary("cartesian_shape")),
+
+        // version: convert to string
+        buildEntry(new BlockConverter.TransformedBytesRef("version", MinorType.VARCHAR, ValueConversions::versionToString)),
+
+        // _source: json
+        // TODO: support also CBOR and SMILE with an additional formatting parameter
+        buildEntry(new BlockConverter.TransformedBytesRef("_source", MinorType.VARCHAR, ValueConversions::sourceToJson))
+    );
+
+    private static Map.Entry<String, BlockConverter> buildEntry(BlockConverter converter) {
+        return Map.entry(converter.esqlType(), converter);
+    }
+}

+ 452 - 0
x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/BlockConverter.java

@@ -0,0 +1,452 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.arrow;
+
+import org.apache.arrow.memory.ArrowBuf;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.common.io.stream.RecyclerBytesStreamOutput;
+import org.elasticsearch.compute.data.Block;
+import org.elasticsearch.compute.data.BooleanBlock;
+import org.elasticsearch.compute.data.BytesRefBlock;
+import org.elasticsearch.compute.data.DoubleBlock;
+import org.elasticsearch.compute.data.IntBlock;
+import org.elasticsearch.compute.data.LongBlock;
+
+import java.io.IOException;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiFunction;
+
+public abstract class BlockConverter {
+
+    private final FieldType fieldType;
+    private final String esqlType;
+
+    protected BlockConverter(String esqlType, Types.MinorType minorType) {
+        // Add the exact ESQL type as field metadata
+        var meta = Map.of("elastic:type", esqlType);
+        this.fieldType = new FieldType(true, minorType.getType(), null, meta);
+        this.esqlType = esqlType;
+    }
+
+    public final String esqlType() {
+        return this.esqlType;
+    }
+
+    public final FieldType arrowFieldType() {
+        return this.fieldType;
+    }
+
+    // Block.nullValuesCount was more efficient but was removed in https://github.com/elastic/elasticsearch/pull/108916
+    protected int nullValuesCount(Block block) {
+        if (block.mayHaveNulls() == false) {
+            return 0;
+        }
+
+        if (block.areAllValuesNull()) {
+            return block.getPositionCount();
+        }
+
+        int count = 0;
+        for (int i = 0; i < block.getPositionCount(); i++) {
+            if (block.isNull(i)) {
+                count++;
+            }
+        }
+        return count;
+    }
+
+    public interface BufWriter {
+        long write(RecyclerBytesStreamOutput out) throws IOException;
+    }
+
+    /**
+     * Convert a block into Arrow buffers.
+     * @param block the ESQL block
+     * @param bufs arrow buffers, used to track sizes
+     * @param bufWriters buffer writers, that will do the actual work of writing the data
+     */
+    public abstract void convert(Block block, List<ArrowBuf> bufs, List<BufWriter> bufWriters);
+
+    /**
+     * Conversion of Double blocks
+     */
+    public static class AsFloat64 extends BlockConverter {
+
+        public AsFloat64(String esqlType) {
+            super(esqlType, Types.MinorType.FLOAT8);
+        }
+
+        @Override
+        public void convert(Block b, List<ArrowBuf> bufs, List<BufWriter> bufWriters) {
+            DoubleBlock block = (DoubleBlock) b;
+
+            accumulateVectorValidity(bufs, bufWriters, block);
+
+            bufs.add(dummyArrowBuf(vectorLength(block)));
+            bufWriters.add(out -> {
+                if (block.areAllValuesNull()) {
+                    return BlockConverter.writeZeroes(out, vectorLength(block));
+                }
+
+                // TODO could we "just" get the memory of the array and dump it?
+                int count = block.getPositionCount();
+                for (int i = 0; i < count; i++) {
+                    out.writeDoubleLE(block.getDouble(i));
+                }
+                return vectorLength(block);
+            });
+        }
+
+        private static int vectorLength(DoubleBlock b) {
+            return Double.BYTES * b.getPositionCount();
+        }
+    }
+
+    /**
+     * Conversion of Int blocks
+     */
+    public static class AsInt32 extends BlockConverter {
+
+        public AsInt32(String esqlType) {
+            super(esqlType, Types.MinorType.INT);
+        }
+
+        @Override
+        public void convert(Block b, List<ArrowBuf> bufs, List<BufWriter> bufWriters) {
+            IntBlock block = (IntBlock) b;
+
+            accumulateVectorValidity(bufs, bufWriters, block);
+
+            bufs.add(dummyArrowBuf(vectorLength(block)));
+            bufWriters.add(out -> {
+                if (block.areAllValuesNull()) {
+                    return BlockConverter.writeZeroes(out, vectorLength(block));
+                }
+
+                // TODO could we "just" get the memory of the array and dump it?
+                int count = block.getPositionCount();
+                for (int i = 0; i < count; i++) {
+                    out.writeIntLE(block.getInt(i));
+                }
+                return vectorLength(block);
+            });
+        }
+
+        private static int vectorLength(IntBlock b) {
+            return Integer.BYTES * b.getPositionCount();
+        }
+    }
+
+    /**
+     * Conversion of Long blocks
+     */
+    public static class AsInt64 extends BlockConverter {
+        public AsInt64(String esqlType) {
+            this(esqlType, Types.MinorType.BIGINT);
+        }
+
+        protected AsInt64(String esqlType, Types.MinorType minorType) {
+            super(esqlType, minorType);
+        }
+
+        @Override
+        public void convert(Block b, List<ArrowBuf> bufs, List<BufWriter> bufWriters) {
+            LongBlock block = (LongBlock) b;
+            accumulateVectorValidity(bufs, bufWriters, block);
+
+            bufs.add(dummyArrowBuf(vectorLength(block)));
+            bufWriters.add(out -> {
+                if (block.areAllValuesNull()) {
+                    return BlockConverter.writeZeroes(out, vectorLength(block));
+                }
+
+                // TODO could we "just" get the memory of the array and dump it?
+                int count = block.getPositionCount();
+                for (int i = 0; i < count; i++) {
+                    out.writeLongLE(block.getLong(i));
+                }
+                return vectorLength(block);
+            });
+        }
+
+        private static int vectorLength(LongBlock b) {
+            return Long.BYTES * b.getPositionCount();
+        }
+    }
+
+    /**
+     * Conversion of Boolean blocks
+     */
+    public static class AsBoolean extends BlockConverter {
+        public AsBoolean(String esqlType) {
+            super(esqlType, Types.MinorType.BIT);
+        }
+
+        @Override
+        public void convert(Block b, List<ArrowBuf> bufs, List<BufWriter> bufWriters) {
+            BooleanBlock block = (BooleanBlock) b;
+            accumulateVectorValidity(bufs, bufWriters, block);
+
+            bufs.add(dummyArrowBuf(vectorLength(block)));
+            bufWriters.add(out -> {
+                int count = block.getPositionCount();
+                BitSet bits = new BitSet();
+
+                // Only set the bits that are true, writeBitSet will take
+                // care of adding zero bytes if needed.
+                if (block.areAllValuesNull() == false) {
+                    for (int i = 0; i < count; i++) {
+                        if (block.getBoolean(i)) {
+                            bits.set(i);
+                        }
+                    }
+                }
+
+                return BlockConverter.writeBitSet(out, bits, count);
+            });
+        }
+
+        private static int vectorLength(BooleanBlock b) {
+            return BlockConverter.bitSetLength(b.getPositionCount());
+        }
+    }
+
+    /**
+     * Conversion of ByteRef blocks
+     */
+    public static class BytesRefConverter extends BlockConverter {
+
+        public BytesRefConverter(String esqlType, Types.MinorType minorType) {
+            super(esqlType, minorType);
+        }
+
+        @Override
+        public void convert(Block b, List<ArrowBuf> bufs, List<BufWriter> bufWriters) {
+            BytesRefBlock block = (BytesRefBlock) b;
+
+            BlockConverter.accumulateVectorValidity(bufs, bufWriters, block);
+
+            // Offsets vector
+            bufs.add(dummyArrowBuf(offsetVectorLength(block)));
+
+            bufWriters.add(out -> {
+                if (block.areAllValuesNull()) {
+                    var count = block.getPositionCount() + 1;
+                    for (int i = 0; i < count; i++) {
+                        out.writeIntLE(0);
+                    }
+                    return offsetVectorLength(block);
+                }
+
+                // TODO could we "just" get the memory of the array and dump it?
+                BytesRef scratch = new BytesRef();
+                int offset = 0;
+                for (int i = 0; i < block.getPositionCount(); i++) {
+                    out.writeIntLE(offset);
+                    // FIXME: add a ByteRefsVector.getLength(position): there are some cases
+                    // where getBytesRef will allocate, which isn't needed here.
+                    BytesRef v = block.getBytesRef(i, scratch);
+
+                    offset += v.length;
+                }
+                out.writeIntLE(offset);
+                return offsetVectorLength(block);
+            });
+
+            // Data vector
+            bufs.add(BlockConverter.dummyArrowBuf(dataVectorLength(block)));
+
+            bufWriters.add(out -> {
+                if (block.areAllValuesNull()) {
+                    return 0;
+                }
+
+                // TODO could we "just" get the memory of the array and dump it?
+                BytesRef scratch = new BytesRef();
+                long length = 0;
+                for (int i = 0; i < block.getPositionCount(); i++) {
+                    BytesRef v = block.getBytesRef(i, scratch);
+
+                    out.write(v.bytes, v.offset, v.length);
+                    length += v.length;
+                }
+                return length;
+            });
+        }
+
+        private static int offsetVectorLength(BytesRefBlock block) {
+            return Integer.BYTES * (block.getPositionCount() + 1);
+        }
+
+        private int dataVectorLength(BytesRefBlock block) {
+            if (block.areAllValuesNull()) {
+                return 0;
+            }
+
+            // TODO we can probably get the length from the vector without all this sum
+
+            int length = 0;
+            BytesRef scratch = new BytesRef();
+            for (int i = 0; i < block.getPositionCount(); i++) {
+                BytesRef v = block.getBytesRef(i, scratch);
+                length += v.length;
+            }
+            return length;
+        }
+    }
+
+    /**
+     * Conversion of ByteRefs where each value is itself converted to a different format.
+     */
+    public static class TransformedBytesRef extends BytesRefConverter {
+
+        private final BiFunction<BytesRef, BytesRef, BytesRef> valueConverter;
+
+        /**
+         *
+         * @param esqlType ESQL type name
+         * @param minorType Arrow type
+         * @param valueConverter a function that takes (value, scratch) input parameters and returns the transformed value
+         */
+        public TransformedBytesRef(String esqlType, Types.MinorType minorType, BiFunction<BytesRef, BytesRef, BytesRef> valueConverter) {
+            super(esqlType, minorType);
+            this.valueConverter = valueConverter;
+        }
+
+        @Override
+        public void convert(Block b, List<ArrowBuf> bufs, List<BufWriter> bufWriters) {
+            BytesRefBlock block = (BytesRefBlock) b;
+            try (BytesRefBlock transformed = transformValues(block)) {
+                super.convert(transformed, bufs, bufWriters);
+            }
+        }
+
+        /**
+         * Creates a new BytesRefBlock by applying the value converter to each non null and non empty value
+         */
+        private BytesRefBlock transformValues(BytesRefBlock block) {
+            try (BytesRefBlock.Builder builder = block.blockFactory().newBytesRefBlockBuilder(block.getPositionCount())) {
+                BytesRef scratch = new BytesRef();
+                for (int i = 0; i < block.getPositionCount(); i++) {
+                    if (block.isNull(i)) {
+                        builder.appendNull();
+                    } else {
+                        BytesRef bytes = block.getBytesRef(i, scratch);
+                        if (bytes.length != 0) {
+                            bytes = valueConverter.apply(bytes, scratch);
+                        }
+                        builder.appendBytesRef(bytes);
+                    }
+                }
+                return builder.build();
+            }
+        }
+    }
+
+    public static class AsVarChar extends BytesRefConverter {
+        public AsVarChar(String esqlType) {
+            super(esqlType, Types.MinorType.VARCHAR);
+        }
+    }
+
+    public static class AsVarBinary extends BytesRefConverter {
+        public AsVarBinary(String esqlType) {
+            super(esqlType, Types.MinorType.VARBINARY);
+        }
+    }
+
+    public static class AsNull extends BlockConverter {
+        public AsNull(String esqlType) {
+            super(esqlType, Types.MinorType.NULL);
+        }
+
+        @Override
+        public void convert(Block block, List<ArrowBuf> bufs, List<BufWriter> bufWriters) {
+            // Null vector in arrow has no associated buffers
+            // See https://arrow.apache.org/docs/format/Columnar.html#null-layout
+        }
+    }
+
+    // Create a dummy ArrowBuf used for size accounting purposes.
+    private static ArrowBuf dummyArrowBuf(long size) {
+        return new ArrowBuf(null, null, 0, 0).writerIndex(size);
+    }
+
+    // Length in bytes of a validity buffer
+    private static int bitSetLength(int totalValues) {
+        return (totalValues + 7) / 8;
+    }
+
+    private static void accumulateVectorValidity(List<ArrowBuf> bufs, List<BufWriter> bufWriters, Block b) {
+        bufs.add(dummyArrowBuf(bitSetLength(b.getPositionCount())));
+        bufWriters.add(out -> {
+            if (b.mayHaveNulls() == false) {
+                return writeAllTrueValidity(out, b.getPositionCount());
+            } else if (b.areAllValuesNull()) {
+                return writeAllFalseValidity(out, b.getPositionCount());
+            } else {
+                return writeValidities(out, b);
+            }
+        });
+    }
+
+    private static long writeAllTrueValidity(RecyclerBytesStreamOutput out, int valueCount) {
+        int allOnesCount = valueCount / 8;
+        for (int i = 0; i < allOnesCount; i++) {
+            out.writeByte((byte) 0xff);
+        }
+        int remaining = valueCount % 8;
+        if (remaining == 0) {
+            return allOnesCount;
+        }
+        out.writeByte((byte) ((1 << remaining) - 1));
+        return allOnesCount + 1;
+    }
+
+    private static long writeAllFalseValidity(RecyclerBytesStreamOutput out, int valueCount) {
+        int count = bitSetLength(valueCount);
+        for (int i = 0; i < count; i++) {
+            out.writeByte((byte) 0x00);
+        }
+        return count;
+    }
+
+    private static long writeValidities(RecyclerBytesStreamOutput out, Block block) {
+        int valueCount = block.getPositionCount();
+        BitSet bits = new BitSet(valueCount);
+        for (int i = 0; i < block.getPositionCount(); i++) {
+            if (block.isNull(i) == false) {
+                bits.set(i);
+            }
+        }
+        return writeBitSet(out, bits, valueCount);
+    }
+
+    private static long writeBitSet(RecyclerBytesStreamOutput out, BitSet bits, int bitCount) {
+        byte[] bytes = bits.toByteArray();
+        out.writeBytes(bytes, 0, bytes.length);
+
+        // toByteArray will return bytes up to the last bit set. It may therefore
+        // have a length lower than what is needed to actually store bitCount bits.
+        int expectedLength = bitSetLength(bitCount);
+        writeZeroes(out, expectedLength - bytes.length);
+
+        return expectedLength;
+    }
+
+    private static long writeZeroes(RecyclerBytesStreamOutput out, int byteCount) {
+        for (int i = 0; i < byteCount; i++) {
+            out.writeByte((byte) 0);
+        }
+        return byteCount;
+    }
+}

+ 80 - 0
x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/ValueConversions.java

@@ -0,0 +1,80 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.arrow;
+
+import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.common.bytes.BytesArray;
+import org.elasticsearch.common.xcontent.XContentHelper;
+import org.elasticsearch.xcontent.XContentType;
+import org.elasticsearch.xpack.versionfield.Version;
+
+import java.io.IOException;
+
+/**
+ * Utilities to convert some of byte-encoded ESQL values into to a format more suitable
+ * for Arrow output.
+ */
+public class ValueConversions {
+
+    /**
+     * Shorten ipv6-mapped ipv4 IP addresses to 4 bytes
+     */
+    public static BytesRef shortenIpV4Addresses(BytesRef value, BytesRef scratch) {
+        // Same logic as sun.net.util.IPAddressUtil#isIPv4MappedAddress
+        // See https://datatracker.ietf.org/doc/html/rfc4291#section-2.5.5.2
+        if (value.length == 16) {
+            int pos = value.offset;
+            byte[] bytes = value.bytes;
+            boolean isIpV4 = bytes[pos++] == 0
+                && bytes[pos++] == 0
+                && bytes[pos++] == 0
+                && bytes[pos++] == 0
+                && bytes[pos++] == 0
+                && bytes[pos++] == 0
+                && bytes[pos++] == 0
+                && bytes[pos++] == 0
+                && bytes[pos++] == 0
+                && bytes[pos++] == 0
+                && bytes[pos++] == (byte) 0xFF
+                && bytes[pos] == (byte) 0xFF;
+
+            if (isIpV4) {
+                scratch.bytes = value.bytes;
+                scratch.offset = value.offset + 12;
+                scratch.length = 4;
+                return scratch;
+            }
+        }
+        return value;
+    }
+
+    /**
+     * Convert binary-encoded versions to strings
+     */
+    public static BytesRef versionToString(BytesRef value, BytesRef scratch) {
+        return new BytesRef(new Version(value).toString());
+    }
+
+    /**
+     * Convert any xcontent source to json
+     */
+    public static BytesRef sourceToJson(BytesRef value, BytesRef scratch) {
+        try {
+            var valueArray = new BytesArray(value);
+            XContentType xContentType = XContentHelper.xContentType(valueArray);
+            if (xContentType == XContentType.JSON) {
+                return value;
+            } else {
+                String json = XContentHelper.convertToJson(valueArray, false, xContentType);
+                return new BytesRef(json);
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+}

+ 600 - 0
x-pack/plugin/esql/arrow/src/test/java/org/elasticsearch/xpack/esql/arrow/ArrowResponseTests.java

@@ -0,0 +1,600 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.arrow;
+
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMilliVector;
+import org.apache.arrow.vector.UInt8Vector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.ipc.ArrowStreamReader;
+import org.apache.arrow.vector.util.VectorSchemaRootAppender;
+import org.apache.lucene.document.InetAddressPoint;
+import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.common.TriFunction;
+import org.elasticsearch.common.breaker.NoopCircuitBreaker;
+import org.elasticsearch.common.bytes.BytesReference;
+import org.elasticsearch.common.bytes.CompositeBytesReference;
+import org.elasticsearch.common.util.BigArrays;
+import org.elasticsearch.compute.data.Block;
+import org.elasticsearch.compute.data.BlockFactory;
+import org.elasticsearch.compute.data.BooleanBlock;
+import org.elasticsearch.compute.data.BytesRefBlock;
+import org.elasticsearch.compute.data.DoubleBlock;
+import org.elasticsearch.compute.data.IntBlock;
+import org.elasticsearch.compute.data.IntVectorBlock;
+import org.elasticsearch.compute.data.LongBlock;
+import org.elasticsearch.compute.data.Page;
+import org.elasticsearch.test.ESTestCase;
+import org.elasticsearch.transport.BytesRefRecycler;
+import org.elasticsearch.xpack.versionfield.Version;
+import org.junit.AfterClass;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+public class ArrowResponseTests extends ESTestCase {
+
+    private static final BlockFactory BLOCK_FACTORY = BlockFactory.getInstance(
+        new NoopCircuitBreaker("test-noop"),
+        BigArrays.NON_RECYCLING_INSTANCE
+    );
+
+    private static final RootAllocator ALLOCATOR = new RootAllocator();
+
+    @AfterClass
+    public static void afterClass() throws Exception {
+        ALLOCATOR.close();
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Value creation, getters for ESQL and Arrow
+
+    static final ValueType INTEGER_VALUES = new ValueTypeImpl<IntBlock.Builder, IntBlock, IntVector>(
+        factory -> factory.newIntBlockBuilder(0),
+        block -> block.appendInt(randomInt()),
+        (block, i, scratch) -> block.getInt(i),
+        IntVector::get
+    );
+
+    static final ValueType LONG_VALUES = new ValueTypeImpl<LongBlock.Builder, LongBlock, BigIntVector>(
+        factory -> factory.newLongBlockBuilder(0),
+        block -> block.appendLong(randomLong()),
+        (block, i, scratch) -> block.getLong(i),
+        BigIntVector::get
+    );
+
+    static final ValueType ULONG_VALUES = new ValueTypeImpl<LongBlock.Builder, LongBlock, UInt8Vector>(
+        factory -> factory.newLongBlockBuilder(0),
+        block -> block.appendLong(randomLong()),
+        (block, i, scratch) -> block.getLong(i),
+        UInt8Vector::get
+    );
+
+    static final ValueType DATE_VALUES = new ValueTypeImpl<LongBlock.Builder, LongBlock, TimeStampMilliVector>(
+        factory -> factory.newLongBlockBuilder(0),
+        block -> block.appendLong(randomLong()),
+        (block, i, scratch) -> block.getLong(i),
+        TimeStampMilliVector::get
+    );
+
+    static final ValueType DOUBLE_VALUES = new ValueTypeImpl<DoubleBlock.Builder, DoubleBlock, Float8Vector>(
+        factory -> factory.newDoubleBlockBuilder(0),
+        block -> block.appendDouble(randomDouble()),
+        (block, i, scratch) -> block.getDouble(i),
+        Float8Vector::get
+    );
+
+    static final ValueType BOOLEAN_VALUES = new ValueTypeImpl<BooleanBlock.Builder, BooleanBlock, BitVector>(
+        factory -> factory.newBooleanBlockBuilder(0),
+        block -> block.appendBoolean(randomBoolean()),
+        (b, i, s) -> b.getBoolean(i),
+        (v, i) -> v.get(i) != 0 // Arrow's BitVector returns 0 or 1
+    );
+
+    static final ValueType TEXT_VALUES = new ValueTypeImpl<BytesRefBlock.Builder, BytesRefBlock, VarCharVector>(
+        factory -> factory.newBytesRefBlockBuilder(0),
+        block -> block.appendBytesRef(new BytesRef("🚀" + randomAlphaOfLengthBetween(1, 20))),
+        (b, i, s) -> b.getBytesRef(i, s).utf8ToString(),
+        (v, i) -> new String(v.get(i), StandardCharsets.UTF_8)
+    );
+
+    static final ValueType SOURCE_VALUES = new ValueTypeImpl<BytesRefBlock.Builder, BytesRefBlock, VarCharVector>(
+        factory -> factory.newBytesRefBlockBuilder(0),
+        // Use a constant value, conversion is tested separately
+        block -> block.appendBytesRef(new BytesRef("{\"foo\": 42}")),
+        (b, i, s) -> b.getBytesRef(i, s).utf8ToString(),
+        (v, i) -> new String(v.get(i), StandardCharsets.UTF_8)
+    );
+
+    static final ValueType IP_VALUES = new ValueTypeImpl<BytesRefBlock.Builder, BytesRefBlock, VarBinaryVector>(
+        factory -> factory.newBytesRefBlockBuilder(0),
+        block -> {
+            byte[] addr = InetAddressPoint.encode(randomIp(randomBoolean()));
+            assertEquals(16, addr.length); // Make sure all is ipv6-mapped
+            block.appendBytesRef(new BytesRef(addr));
+        },
+        (b, i, s) -> ValueConversions.shortenIpV4Addresses(b.getBytesRef(i, s), new BytesRef()),
+        (v, i) -> new BytesRef(v.get(i))
+    );
+
+    static final ValueType BINARY_VALUES = new ValueTypeImpl<BytesRefBlock.Builder, BytesRefBlock, VarBinaryVector>(
+        factory -> factory.newBytesRefBlockBuilder(0),
+        block -> block.appendBytesRef(new BytesRef(randomByteArrayOfLength(randomIntBetween(1, 100)))),
+        BytesRefBlock::getBytesRef,
+        (v, i) -> new BytesRef(v.get(i))
+    );
+
+    static final ValueType VERSION_VALUES = new ValueTypeImpl<BytesRefBlock.Builder, BytesRefBlock, VarCharVector>(
+        factory -> factory.newBytesRefBlockBuilder(0),
+        block -> block.appendBytesRef(new Version(between(0, 100) + "." + between(0, 100) + "." + between(0, 100)).toBytesRef()),
+        (b, i, s) -> new Version(b.getBytesRef(i, s)).toString(),
+        (v, i) -> new String(v.get(i), StandardCharsets.UTF_8)
+    );
+
+    static final ValueType NULL_VALUES = new ValueTypeImpl<Block.Builder, Block, FieldVector>(
+        factory -> factory.newBytesRefBlockBuilder(0),
+        Block.Builder::appendNull,
+        (b, i, s) -> b.isNull(i) ? null : "non-null in block",
+        (v, i) -> v.isNull(i) ? null : "non-null in vector"
+    );
+
+    static final Map<String, ValueType> VALUE_TYPES = Map.ofEntries(
+        Map.entry("integer", INTEGER_VALUES),
+        Map.entry("counter_integer", INTEGER_VALUES),
+        Map.entry("long", LONG_VALUES),
+        Map.entry("counter_long", LONG_VALUES),
+        Map.entry("unsigned_long", ULONG_VALUES),
+        Map.entry("double", DOUBLE_VALUES),
+        Map.entry("counter_double", DOUBLE_VALUES),
+
+        Map.entry("text", TEXT_VALUES),
+        Map.entry("keyword", TEXT_VALUES),
+
+        Map.entry("boolean", BOOLEAN_VALUES),
+        Map.entry("date", DATE_VALUES),
+        Map.entry("ip", IP_VALUES),
+        Map.entry("version", VERSION_VALUES),
+        Map.entry("_source", SOURCE_VALUES),
+
+        Map.entry("null", NULL_VALUES),
+        Map.entry("unsupported", NULL_VALUES),
+
+        // All geo types just pass-through WKB, use random binary data
+        Map.entry("geo_point", BINARY_VALUES),
+        Map.entry("geo_shape", BINARY_VALUES),
+        Map.entry("cartesian_point", BINARY_VALUES),
+        Map.entry("cartesian_shape", BINARY_VALUES)
+    );
+
+    // ---------------------------------------------------------------------------------------------
+    // Tests
+
+    public void testTestHarness() {
+        TestColumn testColumn = TestColumn.create("foo", "integer");
+        TestBlock denseBlock = TestBlock.create(BLOCK_FACTORY, testColumn, Density.Dense, 3);
+        TestBlock sparseBlock = TestBlock.create(BLOCK_FACTORY, testColumn, Density.Sparse, 5);
+        TestBlock emptyBlock = TestBlock.create(BLOCK_FACTORY, testColumn, Density.Empty, 7);
+
+        // Test that density works as expected
+        assertTrue(denseBlock.block instanceof IntVectorBlock);
+        assertEquals("IntArrayBlock", sparseBlock.block.getClass().getSimpleName()); // non-public class
+        assertEquals("ConstantNullBlock", emptyBlock.block.getClass().getSimpleName());
+
+        // Test that values iterator scans all pages
+        List<TestPage> pages = Stream.of(denseBlock, sparseBlock, emptyBlock).map(b -> new TestPage(List.of(b))).toList();
+        TestCase tc = new TestCase(List.of(testColumn), pages);
+        EsqlValuesIterator valuesIterator = new EsqlValuesIterator(tc, 0);
+        int count = 0;
+        while (valuesIterator.hasNext()) {
+            valuesIterator.next();
+            count++;
+        }
+        assertEquals(3 + 5 + 7, count);
+
+        // Test that we have value types for all types
+        List<String> converters = new ArrayList<>(ArrowResponse.ESQL_CONVERTERS.keySet());
+        Collections.sort(converters);
+        List<String> valueTypes = new ArrayList<>(VALUE_TYPES.keySet());
+        Collections.sort(valueTypes);
+        assertEquals("Missing test value types", converters, valueTypes);
+    }
+
+    /**
+     * Test single-column for all types with a mix of dense/sparse/empty pages
+     */
+    public void testSingleColumn() throws IOException {
+        for (var type : VALUE_TYPES.keySet()) {
+            TestColumn testColumn = new TestColumn("foo", type, VALUE_TYPES.get(type));
+            List<TestPage> pages = new ArrayList<>();
+
+            for (var density : Density.values()) {
+                TestBlock testBlock = TestBlock.create(BLOCK_FACTORY, testColumn, density, 10);
+                TestPage testPage = new TestPage(List.of(testBlock));
+                pages.add(testPage);
+            }
+            TestCase testCase = new TestCase(List.of(testColumn), pages);
+
+            compareEsqlAndArrow(testCase);
+        }
+    }
+
+    public void testSingleBlock() throws IOException {
+        // Simple test to easily focus on a specific type & density
+        String type = "text";
+        Density density = Density.Dense;
+
+        TestColumn testColumn = new TestColumn("foo", type, VALUE_TYPES.get(type));
+        List<TestPage> pages = new ArrayList<>();
+
+        TestBlock testBlock = TestBlock.create(BLOCK_FACTORY, testColumn, density, 10);
+        TestPage testPage = new TestPage(List.of(testBlock));
+        pages.add(testPage);
+
+        TestCase testCase = new TestCase(List.of(testColumn), pages);
+
+        compareEsqlAndArrow(testCase);
+    }
+
+    /**
+     * Test that multivalued arrays are rejected
+     */
+    public void testMultivaluedField() throws IOException {
+        IntBlock.Builder builder = BLOCK_FACTORY.newIntBlockBuilder(0);
+        builder.appendInt(42);
+        builder.appendNull();
+        builder.beginPositionEntry();
+        builder.appendInt(44);
+        builder.appendInt(45);
+        builder.endPositionEntry();
+        builder.appendInt(46);
+        IntBlock block = builder.build();
+
+        // Consistency check
+        assertTrue(block.mayHaveMultivaluedFields());
+        assertEquals(0, block.getFirstValueIndex(0));
+        assertEquals(1, block.getValueCount(0));
+
+        // null values still use one position in the array
+        assertEquals(0, block.getValueCount(1));
+        assertEquals(1, block.getFirstValueIndex(1));
+        assertTrue(block.isNull(1));
+        assertEquals(0, block.getInt(1));
+
+        assertEquals(2, block.getFirstValueIndex(2));
+        assertEquals(2, block.getValueCount(2));
+        assertEquals(2, block.getFirstValueIndex(2));
+        assertEquals(45, block.getInt(block.getFirstValueIndex(2) + 1));
+
+        assertEquals(4, block.getFirstValueIndex(3));
+
+        var column = TestColumn.create("some-field", "integer");
+        TestCase testCase = new TestCase(List.of(column), List.of(new TestPage(List.of(new TestBlock(column, block, Density.Dense)))));
+
+        IllegalArgumentException exc = assertThrows(IllegalArgumentException.class, () -> compareEsqlAndArrow(testCase));
+
+        assertEquals("ES|QL response field [some-field] is multi-valued. This isn't supported yet by the Arrow format", exc.getMessage());
+
+    }
+
+    /**
+     * Test a random set of types/columns/pages/densities
+     */
+    public void testRandomTypesAndSize() throws IOException {
+
+        // Shuffle types to randomize their succession in the Arrow stream
+        List<String> types = new ArrayList<>(VALUE_TYPES.keySet());
+        Collections.shuffle(types, random());
+
+        List<TestColumn> columns = types.stream().map(type -> TestColumn.create("col-" + type, type)).toList();
+
+        List<TestPage> pages = IntStream
+            // 1 to 10 pages of random density and 1 to 1000 values
+            .range(0, randomIntBetween(1, 100))
+            .mapToObj(i -> TestPage.create(BLOCK_FACTORY, columns))
+            .toList();
+
+        TestCase testCase = new TestCase(columns, pages);
+        // System.out.println(testCase);
+        // for (TestPage page: pages) {
+        // System.out.println(page);
+        // }
+
+        compareEsqlAndArrow(testCase);
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Test harness
+
+    private void compareEsqlAndArrow(TestCase testCase) throws IOException {
+        try (VectorSchemaRoot arrowVectors = toArrowVectors(testCase)) {
+            compareEsqlAndArrow(testCase, arrowVectors);
+        }
+    }
+
+    private void compareEsqlAndArrow(TestCase testCase, VectorSchemaRoot root) {
+        for (int i = 0; i < testCase.columns.size(); i++) {
+
+            // Check esql type in the metadata
+            var metadata = root.getSchema().getFields().get(i).getMetadata();
+            assertEquals(testCase.columns.get(i).type, metadata.get("elastic:type"));
+
+            // Check values
+            var esqlValuesIterator = new EsqlValuesIterator(testCase, i);
+            var arrowValuesIterator = new ArrowValuesIterator(testCase, root, i);
+
+            while (esqlValuesIterator.hasNext() && arrowValuesIterator.hasNext()) {
+                assertEquals(esqlValuesIterator.next(), arrowValuesIterator.next());
+            }
+
+            // Make sure we entirely consumed both sides.
+            assertFalse(esqlValuesIterator.hasNext());
+            assertFalse(arrowValuesIterator.hasNext());
+        }
+    }
+
+    private VectorSchemaRoot toArrowVectors(TestCase testCase) throws IOException {
+        ArrowResponse response = new ArrowResponse(
+            testCase.columns.stream().map(c -> new ArrowResponse.Column(c.type, c.name)).toList(),
+            testCase.pages.stream().map(p -> new Page(p.blocks.stream().map(b -> b.block).toArray(Block[]::new))).toList()
+        );
+
+        assertEquals("application/vnd.apache.arrow.stream", response.getResponseContentTypeString());
+
+        BytesReference bytes = serializeBlocksDirectly(response);
+        try (
+            ArrowStreamReader reader = new ArrowStreamReader(bytes.streamInput(), ALLOCATOR);
+            VectorSchemaRoot readerRoot = reader.getVectorSchemaRoot();
+        ) {
+            VectorSchemaRoot root = VectorSchemaRoot.create(readerRoot.getSchema(), ALLOCATOR);
+            root.allocateNew();
+
+            while (reader.loadNextBatch()) {
+                VectorSchemaRootAppender.append(root, readerRoot);
+            }
+
+            return root;
+        }
+    }
+
+    /**
+     * An iterator over values of a column across all pages.
+     */
+    static class EsqlValuesIterator implements Iterator<Object> {
+        private final int fieldPos;
+        private final ValueType type;
+        private final BytesRef scratch = new BytesRef();
+        private final Iterator<TestPage> pages;
+
+        private TestPage page;
+        private int position;
+
+        EsqlValuesIterator(TestCase testCase, int column) {
+            this.fieldPos = column;
+            this.type = testCase.columns.get(column).valueType;
+            this.position = 0;
+            this.pages = testCase.pages.iterator();
+            this.page = pages.next();
+        }
+
+        @Override
+        public boolean hasNext() {
+            return page != null;
+        }
+
+        @Override
+        public Object next() {
+            if (page == null) {
+                throw new NoSuchElementException();
+            }
+            Block block = page.blocks.get(fieldPos).block;
+            Object result = block.isNull(position) ? null : type.valueAt(block, position, scratch);
+            position++;
+            if (position >= block.getPositionCount()) {
+                position = 0;
+                page = pages.hasNext() ? pages.next() : null;
+            }
+            return result;
+        }
+    }
+
+    static class ArrowValuesIterator implements Iterator<Object> {
+        private final ValueType type;
+        private ValueVector vector;
+        private int position;
+
+        ArrowValuesIterator(TestCase testCase, VectorSchemaRoot root, int column) {
+            this(root.getVector(column), testCase.columns.get(column).valueType);
+        }
+
+        ArrowValuesIterator(ValueVector vector, ValueType type) {
+            this.vector = vector;
+            this.type = type;
+        }
+
+        @Override
+        public boolean hasNext() {
+            return vector != null;
+        }
+
+        @Override
+        public Object next() {
+            if (vector == null) {
+                throw new NoSuchElementException();
+            }
+            Object result = vector.isNull(position) ? null : type.valueAt(vector, position);
+            position++;
+            if (position >= vector.getValueCount()) {
+                vector = null;
+            }
+            return result;
+        }
+    }
+
+    private BytesReference serializeBlocksDirectly(ArrowResponse body) throws IOException {
+        // Ensure there's a single part, this will fail if we ever change it.
+        assertTrue(body.isLastPart());
+
+        List<BytesReference> ourEncoding = new ArrayList<>();
+        int page = 0;
+        while (body.isPartComplete() == false) {
+            ourEncoding.add(body.encodeChunk(1500, BytesRefRecycler.NON_RECYCLING_INSTANCE));
+            page++;
+        }
+        return CompositeBytesReference.of(ourEncoding.toArray(BytesReference[]::new));
+    }
+
+    record TestCase(List<TestColumn> columns, List<TestPage> pages) {
+        @Override
+        public String toString() {
+            return pages.size() + " pages of " + columns.stream().map(TestColumn::type).collect(Collectors.joining("|"));
+        }
+    }
+
+    record TestColumn(String name, String type, ValueType valueType) {
+        static TestColumn create(String name, String type) {
+            return new TestColumn(name, type, VALUE_TYPES.get(type));
+        }
+    }
+
+    record TestPage(List<TestBlock> blocks) {
+
+        static TestPage create(BlockFactory factory, List<TestColumn> columns) {
+            int size = randomIntBetween(1, 1000);
+            return new TestPage(columns.stream().map(column -> TestBlock.create(factory, column, size)).toList());
+        }
+
+        @Override
+        public String toString() {
+            return blocks.get(0).block.getPositionCount()
+                + " items - "
+                + blocks.stream().map(b -> b.density.toString()).collect(Collectors.joining("|"));
+        }
+    }
+
+    record TestBlock(TestColumn column, Block block, Density density) {
+
+        static TestBlock create(BlockFactory factory, TestColumn column, int positions) {
+            return create(factory, column, randomFrom(Density.values()), positions);
+        }
+
+        static TestBlock create(BlockFactory factory, TestColumn column, Density density, int positions) {
+            ValueType valueType = column.valueType();
+            Block block;
+            if (density == Density.Empty) {
+                block = factory.newConstantNullBlock(positions);
+            } else {
+                Block.Builder builder = valueType.createBlockBuilder(factory);
+                int start = 0;
+                if (density == Density.Sparse && positions >= 2) {
+                    // Make sure it's really sparse even if randomness of values may decide otherwise
+                    valueType.addValue(builder, Density.Dense);
+                    valueType.addValue(builder, Density.Empty);
+                    start = 2;
+                }
+                for (int i = start; i < positions; i++) {
+                    valueType.addValue(builder, density);
+                }
+                // Will create an ArrayBlock if there are null values, VectorBlock otherwise
+                block = builder.build();
+            }
+            return new TestBlock(column, block, density);
+        }
+    }
+
+    public enum Density {
+        Empty,
+        Sparse,
+        Dense;
+
+        boolean nextIsNull() {
+            return switch (this) {
+                case Empty -> true;
+                case Sparse -> randomBoolean();
+                case Dense -> false;
+            };
+        }
+    }
+
+    interface ValueType {
+        Block.Builder createBlockBuilder(BlockFactory factory);
+
+        void addValue(Block.Builder builder, Density density);
+
+        Object valueAt(Block block, int position, BytesRef scratch);
+
+        Object valueAt(ValueVector arrowVec, int position);
+    }
+
+    public static class ValueTypeImpl<BlockBT extends Block.Builder, BlockT extends Block, VectorT extends ValueVector>
+        implements
+            ValueType {
+        private final Function<BlockFactory, BlockBT> builderCreator;
+        private final Consumer<BlockBT> valueAdder;
+        private final TriFunction<BlockT, Integer, BytesRef, Object> blockGetter;
+        private final BiFunction<VectorT, Integer, Object> vectorGetter;
+
+        public ValueTypeImpl(
+            Function<BlockFactory, BlockBT> builderCreator,
+            Consumer<BlockBT> valueAdder,
+            TriFunction<BlockT, Integer, BytesRef, Object> blockGetter,
+            BiFunction<VectorT, Integer, Object> vectorGetter
+        ) {
+            this.builderCreator = builderCreator;
+            this.valueAdder = valueAdder;
+            this.blockGetter = blockGetter;
+            this.vectorGetter = vectorGetter;
+        }
+
+        @Override
+        public Block.Builder createBlockBuilder(BlockFactory factory) {
+            return builderCreator.apply(factory);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void addValue(Block.Builder builder, Density density) {
+            if (density.nextIsNull()) {
+                builder.appendNull();
+            } else {
+                valueAdder.accept((BlockBT) builder);
+            }
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public Object valueAt(Block block, int position, BytesRef scratch) {
+            return blockGetter.apply((BlockT) block, position, scratch);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public Object valueAt(ValueVector arrowVec, int position) {
+            return vectorGetter.apply((VectorT) arrowVec, position);
+        }
+    }
+}

+ 84 - 0
x-pack/plugin/esql/arrow/src/test/java/org/elasticsearch/xpack/esql/arrow/ValueConversionsTests.java

@@ -0,0 +1,84 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.arrow;
+
+import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.common.bytes.BytesReference;
+import org.elasticsearch.test.ESTestCase;
+import org.elasticsearch.xcontent.XContentBuilder;
+import org.elasticsearch.xcontent.XContentFactory;
+import org.elasticsearch.xpack.esql.core.util.StringUtils;
+import org.elasticsearch.xpack.versionfield.Version;
+
+public class ValueConversionsTests extends ESTestCase {
+
+    public void testIpConversion() throws Exception {
+        {
+            // ipv6 address
+            BytesRef bytes = StringUtils.parseIP("2a00:1450:4007:818::200e");
+            assertArrayEquals(
+                new byte[] { 0x2a, 0x00, 0x14, 0x50, 0x40, 0x07, 0x08, 0x18, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x20, 0x0e },
+                bytes.bytes
+            );
+
+            BytesRef scratch = new BytesRef();
+            BytesRef bytes2 = ValueConversions.shortenIpV4Addresses(bytes.clone(), scratch);
+            assertEquals(bytes, bytes2);
+        }
+        {
+            // ipv6 mapped ipv4 address
+            BytesRef bytes = StringUtils.parseIP("216.58.214.174");
+            assertArrayEquals(
+                new byte[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, (byte) 0xFF, (byte) 0xFF, (byte) 216, (byte) 58, (byte) 214, (byte) 174 },
+                bytes.bytes
+            );
+
+            BytesRef scratch = new BytesRef();
+            BytesRef bytes2 = ValueConversions.shortenIpV4Addresses(bytes.clone(), scratch);
+
+            assertTrue(new BytesRef(new byte[] { (byte) 216, (byte) 58, (byte) 214, (byte) 174 }).bytesEquals(bytes2));
+
+        }
+    }
+
+    public void testVersionConversion() {
+        String version = "1.2.3-alpha";
+
+        BytesRef bytes = new Version("1.2.3-alpha").toBytesRef();
+
+        BytesRef scratch = new BytesRef();
+        BytesRef bytes2 = ValueConversions.versionToString(bytes, scratch);
+
+        // Some conversion happened
+        assertNotEquals(bytes.length, bytes2.length);
+        assertEquals(version, bytes2.utf8ToString());
+    }
+
+    public void testSourceToJson() throws Exception {
+        BytesRef bytes = new BytesRef("{\"foo\": 42}");
+
+        BytesRef scratch = new BytesRef();
+        BytesRef bytes2 = ValueConversions.sourceToJson(bytes, scratch);
+        // No change, even indentation
+        assertEquals("{\"foo\": 42}", bytes2.utf8ToString());
+    }
+
+    public void testCborSourceToJson() throws Exception {
+        XContentBuilder builder = XContentFactory.cborBuilder();
+        builder.startObject();
+        builder.field("foo", 42);
+        builder.endObject();
+        builder.close();
+        BytesRef bytesRef = BytesReference.bytes(builder).toBytesRef();
+
+        BytesRef scratch = new BytesRef();
+        BytesRef bytes2 = ValueConversions.sourceToJson(bytesRef, scratch);
+        // Converted to JSON
+        assertEquals("{\"foo\":42}", bytes2.utf8ToString());
+    }
+}

+ 13 - 0
x-pack/plugin/esql/arrow/src/test/resources/plugin-security.policy

@@ -0,0 +1,13 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+// Needed by the Arrow memory manager
+grant {
+  permission java.lang.RuntimePermission "accessDeclaredMembers";
+  permission java.lang.reflect.ReflectPermission "suppressAccessChecks";
+  permission java.lang.RuntimePermission "accessClassInPackage.sun.misc";
+};

+ 2 - 0
x-pack/plugin/esql/build.gradle

@@ -25,6 +25,8 @@ dependencies {
   implementation project('compute:ann')
   implementation project(':libs:elasticsearch-dissect')
   implementation project(':libs:elasticsearch-grok')
+  implementation project('arrow')
+
   // Also contains a dummy processor to allow compilation with unused annotations.
   annotationProcessor project('compute:gen')
 

+ 14 - 0
x-pack/plugin/esql/qa/server/single-node/build.gradle

@@ -7,6 +7,19 @@ dependencies {
   javaRestTestImplementation project(xpackModule('esql:qa:testFixtures'))
   javaRestTestImplementation project(xpackModule('esql:qa:server'))
   yamlRestTestImplementation project(xpackModule('esql:qa:server'))
+
+  javaRestTestImplementation('org.apache.arrow:arrow-vector:16.1.0')
+  javaRestTestImplementation('org.apache.arrow:arrow-format:16.1.0')
+  javaRestTestImplementation('org.apache.arrow:arrow-memory-core:16.1.0')
+  javaRestTestImplementation('org.checkerframework:checker-qual:3.42.0')
+  javaRestTestImplementation('com.google.flatbuffers:flatbuffers-java:23.5.26')
+  javaRestTestImplementation("com.fasterxml.jackson.core:jackson-annotations:${versions.jackson}")
+  javaRestTestImplementation("com.fasterxml.jackson.core:jackson-core:${versions.jackson}")
+  javaRestTestImplementation("com.fasterxml.jackson.core:jackson-databind:${versions.jackson}")
+  javaRestTestImplementation("org.slf4j:slf4j-api:${versions.slf4j}")
+  javaRestTestImplementation("org.slf4j:slf4j-nop:${versions.slf4j}")
+  javaRestTestImplementation('org.apache.arrow:arrow-memory-unsafe:16.1.0')
+
   dependencies {
     clusterPlugins project(':plugins:mapper-size')
     clusterPlugins project(':plugins:mapper-murmur3')
@@ -25,6 +38,7 @@ restResources {
 tasks.named('javaRestTest') {
   usesDefaultDistribution()
   maxParallelForks = 1
+  jvmArgs('--add-opens=java.base/java.nio=ALL-UNNAMED')
 }
 
 tasks.named('yamlRestTest') {

+ 242 - 0
x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/ArrowFormatIT.java

@@ -0,0 +1,242 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.qa.single_node;
+
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
+
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.ipc.ArrowStreamReader;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.util.VectorSchemaRootAppender;
+import org.elasticsearch.client.Request;
+import org.elasticsearch.client.Response;
+import org.elasticsearch.test.TestClustersThreadFilter;
+import org.elasticsearch.test.cluster.ElasticsearchCluster;
+import org.elasticsearch.test.rest.ESRestTestCase;
+import org.elasticsearch.xpack.esql.qa.rest.EsqlSpecTestCase;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.ClassRule;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+
+@ThreadLeakFilters(filters = TestClustersThreadFilter.class)
+public class ArrowFormatIT extends ESRestTestCase {
+
+    private static final RootAllocator ALLOCATOR = new RootAllocator();
+
+    @AfterClass
+    public static void afterClass() {
+        ALLOCATOR.close();
+    }
+
+    @ClassRule
+    public static ElasticsearchCluster cluster = Clusters.testCluster();
+
+    @Override
+    protected String getTestRestCluster() {
+        return cluster.getHttpAddresses();
+    }
+
+    @Before
+    @After
+    public void assertRequestBreakerEmpty() throws Exception {
+        EsqlSpecTestCase.assertRequestBreakerEmpty();
+    }
+
+    @Before
+    public void initIndex() throws IOException {
+        Request request = new Request("PUT", "/arrow-test");
+        request.setJsonEntity("""
+            {
+              "mappings": {
+                "properties": {
+                  "value": {
+                    "type": "integer"
+                  },
+                  "description": {
+                    "type": "keyword"
+                  },
+                  "ip": {
+                    "type": "ip"
+                  },
+                  "v": {
+                    "type": "version"
+                  }
+                }
+              }
+            }
+            """);
+        assertEquals(200, client().performRequest(request).getStatusLine().getStatusCode());
+
+        request = new Request("POST", "/_bulk?index=arrow-test&refresh=true");
+        // 4 documents with a null in the middle, leading to 3 ESQL pages and 3 Arrow batches
+        request.setJsonEntity("""
+            {"index": {"_id": "1"}}
+            {"value": 1, "ip": "192.168.0.1", "v": "1.0.1", "description": "number one"}
+            {"index": {"_id": "2"}}
+            {"value": 2, "ip": "192.168.0.2", "v": "1.0.2", "description": "number two"}
+            {"index": {"_id": "3"}}
+            {"value": 3, "ip": "2001:db8::1:0:0:1"}
+            {"index": {"_id": "4"}}
+            {"value": 4, "ip": "::afff:4567:890a", "v": "1.0.4", "description": "number four"}
+            """);
+        assertEquals(200, client().performRequest(request).getStatusLine().getStatusCode());
+    }
+
+    private VectorSchemaRoot esql(String query) throws IOException {
+        Request request = new Request("POST", "/_query?format=arrow");
+        request.setJsonEntity(query);
+        Response response = client().performRequest(request);
+
+        assertEquals("application/vnd.apache.arrow.stream", response.getEntity().getContentType().getValue());
+        return readArrow(response.getEntity().getContent());
+    }
+
+    public void testInteger() throws Exception {
+        try (VectorSchemaRoot root = esql("""
+            {
+                "query": "FROM arrow-test | SORT value | LIMIT 100 | KEEP value"
+            }""")) {
+            List<Field> fields = root.getSchema().getFields();
+            assertEquals(1, fields.size());
+
+            assertValues(root);
+        }
+    }
+
+    public void testString() throws Exception {
+        try (VectorSchemaRoot root = esql("""
+            {
+                "query": "FROM arrow-test | SORT value | LIMIT 100 | KEEP description"
+            }""")) {
+            List<Field> fields = root.getSchema().getFields();
+            assertEquals(1, fields.size());
+
+            assertDescription(root);
+        }
+    }
+
+    public void testIp() throws Exception {
+        try (VectorSchemaRoot root = esql("""
+            {
+                "query": "FROM arrow-test | SORT value | LIMIT 100 | KEEP ip"
+            }""")) {
+            List<Field> fields = root.getSchema().getFields();
+            assertEquals(1, fields.size());
+
+            assertIp(root);
+        }
+    }
+
+    public void testVersion() throws Exception {
+        try (VectorSchemaRoot root = esql("""
+            {
+                "query": "FROM arrow-test | SORT value | LIMIT 100 | KEEP v"
+            }""")) {
+            List<Field> fields = root.getSchema().getFields();
+            assertEquals(1, fields.size());
+
+            assertVersion(root);
+        }
+    }
+
+    public void testEverything() throws Exception {
+        try (VectorSchemaRoot root = esql("""
+            {
+                "query": "FROM arrow-test | SORT value | LIMIT 100"
+            }""")) {
+            List<Field> fields = root.getSchema().getFields();
+            assertEquals(4, fields.size());
+
+            assertDescription(root);
+            assertValues(root);
+            assertIp(root);
+            assertVersion(root);
+        }
+    }
+
+    private VectorSchemaRoot readArrow(InputStream input) throws IOException {
+        try (
+            ArrowStreamReader reader = new ArrowStreamReader(input, ALLOCATOR);
+            VectorSchemaRoot readerRoot = reader.getVectorSchemaRoot();
+        ) {
+            VectorSchemaRoot root = VectorSchemaRoot.create(readerRoot.getSchema(), ALLOCATOR);
+            root.allocateNew();
+
+            while (reader.loadNextBatch()) {
+                VectorSchemaRootAppender.append(root, readerRoot);
+            }
+
+            return root;
+        }
+    }
+
+    private void assertValues(VectorSchemaRoot root) {
+        var valueVector = (IntVector) root.getVector("value");
+        assertEquals(1, valueVector.get(0));
+        assertEquals(2, valueVector.get(1));
+        assertEquals(3, valueVector.get(2));
+        assertEquals(4, valueVector.get(3));
+    }
+
+    private void assertDescription(VectorSchemaRoot root) {
+        var descVector = (VarCharVector) root.getVector("description");
+        assertEquals("number one", descVector.getObject(0).toString());
+        assertEquals("number two", descVector.getObject(1).toString());
+        assertTrue(descVector.isNull(2));
+        assertEquals("number four", descVector.getObject(3).toString());
+    }
+
+    private void assertIp(VectorSchemaRoot root) {
+        // Test data that has been transformed during output (ipV4 truncated to 32bits)
+        var ipVector = (VarBinaryVector) root.getVector("ip");
+        assertArrayEquals(new byte[] { (byte) 192, (byte) 168, 0, 1 }, ipVector.getObject(0));
+        assertArrayEquals(new byte[] { (byte) 192, (byte) 168, 0, 2 }, ipVector.getObject(1));
+        assertArrayEquals(
+            new byte[] { 0x20, 0x01, 0x0d, (byte) 0xb8, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01 },
+            ipVector.getObject(2)
+        );
+        assertArrayEquals(
+            new byte[] {
+                0x00,
+                0x00,
+                0x00,
+                0x00,
+                0x00,
+                0x00,
+                0x00,
+                0x00,
+                0x00,
+                0x00,
+                (byte) 0xaf,
+                (byte) 0xff,
+                0x45,
+                0x67,
+                (byte) 0x89,
+                0x0A },
+            ipVector.getObject(3)
+        );
+    }
+
+    private void assertVersion(VectorSchemaRoot root) {
+        // Version is binary-encoded in ESQL vectors, turned into a string in arrow output
+        var versionVector = (VarCharVector) root.getVector("v");
+        assertEquals("1.0.1", versionVector.getObject(0).toString());
+        assertEquals("1.0.2", versionVector.getObject(1).toString());
+        assertTrue(versionVector.isNull(2));
+        assertEquals("1.0.4", versionVector.getObject(3).toString());
+    }
+}

+ 11 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlResponseListener.java

@@ -9,6 +9,7 @@ package org.elasticsearch.xpack.esql.action;
 
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.core.Releasable;
+import org.elasticsearch.core.Releasables;
 import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.logging.LogManager;
 import org.elasticsearch.logging.Logger;
@@ -19,6 +20,8 @@ import org.elasticsearch.rest.RestResponse;
 import org.elasticsearch.rest.RestStatus;
 import org.elasticsearch.rest.action.RestRefCountedChunkedToXContentListener;
 import org.elasticsearch.xcontent.MediaType;
+import org.elasticsearch.xpack.esql.arrow.ArrowFormat;
+import org.elasticsearch.xpack.esql.arrow.ArrowResponse;
 import org.elasticsearch.xpack.esql.formatter.TextFormat;
 import org.elasticsearch.xpack.esql.plugin.EsqlMediaTypeParser;
 
@@ -135,6 +138,13 @@ public final class EsqlResponseListener extends RestRefCountedChunkedToXContentL
                     ChunkedRestResponseBodyPart.fromTextChunks(format.contentType(restRequest), format.format(restRequest, esqlResponse)),
                     releasable
                 );
+            } else if (mediaType == ArrowFormat.INSTANCE) {
+                ArrowResponse arrowResponse = new ArrowResponse(
+                    // Map here to avoid cyclic dependencies between the arrow subproject and its parent
+                    esqlResponse.columns().stream().map(c -> new ArrowResponse.Column(c.outputType(), c.name())).toList(),
+                    esqlResponse.pages()
+                );
+                restResponse = RestResponse.chunked(RestStatus.OK, arrowResponse, Releasables.wrap(arrowResponse, releasable));
             } else {
                 restResponse = RestResponse.chunked(
                     RestStatus.OK,
@@ -179,4 +189,5 @@ public final class EsqlResponseListener extends RestRefCountedChunkedToXContentL
             listener.onFailure(ex);
         });
     }
+
 }

+ 2 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlMediaTypeParser.java

@@ -13,6 +13,7 @@ import org.elasticsearch.xcontent.MediaTypeRegistry;
 import org.elasticsearch.xcontent.ParsedMediaType;
 import org.elasticsearch.xcontent.XContentType;
 import org.elasticsearch.xpack.esql.action.EsqlQueryRequest;
+import org.elasticsearch.xpack.esql.arrow.ArrowFormat;
 import org.elasticsearch.xpack.esql.formatter.TextFormat;
 
 import java.util.Arrays;
@@ -23,7 +24,7 @@ import static org.elasticsearch.xpack.esql.formatter.TextFormat.URL_PARAM_FORMAT
 public class EsqlMediaTypeParser {
     public static final MediaTypeRegistry<? extends MediaType> MEDIA_TYPE_REGISTRY = new MediaTypeRegistry<>().register(
         XContentType.values()
-    ).register(TextFormat.values());
+    ).register(TextFormat.values()).register(new MediaType[] { ArrowFormat.INSTANCE });
 
     /*
      * Since we support {@link TextFormat} <strong>and</strong>

+ 12 - 0
x-pack/plugin/esql/src/main/plugin-metadata/plugin-security.policy

@@ -0,0 +1,12 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+grant codeBase "${codebase.arrow}" {
+  // Needed for AllocationManagerShim
+  permission java.lang.RuntimePermission "accessDeclaredMembers";
+  permission java.lang.reflect.ReflectPermission "suppressAccessChecks";
+};