From 860b20c63afa64ddaf68addf50fc892693500290 Mon Sep 17 00:00:00 2001 From: Sylvain Wallez Date: Fri, 22 Nov 2024 14:32:27 +0100 Subject: [PATCH 01/19] Add Apache Arrow as a bulk ingestion format --- .idea/inspectionProfiles/Project_Default.xml | 5 +- .../internal/ElasticsearchTestBasePlugin.java | 3 + build-tools-internal/version.properties | 4 +- distribution/src/config/jvm.options | 7 + gradle/build.versions.toml | 2 +- gradle/verification-metadata.xml | 74 ++- libs/arrow/build.gradle | 91 ++++ .../arrow/licenses/arrow-LICENSE.txt | 0 .../arrow/licenses/arrow-NOTICE.txt | 0 libs/arrow/licenses/build.gradle | 49 ++ .../arrow/licenses/checker-qual-LICENSE.txt | 0 .../arrow/licenses/checker-qual-NOTICE.txt | 0 .../licenses/commons-codec-arrow-LICENSE.txt | 0 .../licenses/commons-codec-arrow-NOTICE.txt | 17 + .../error_prone_annotations-LICENSE.txt | 201 +++++++ .../error_prone_annotations-NOTICE.txt | 0 .../licenses/flatbuffers-java-LICENSE.txt | 0 .../licenses/flatbuffers-java-NOTICE.txt | 0 libs/arrow/licenses/jackson-LICENSE.txt | 202 +++++++ libs/arrow/licenses/jackson-NOTICE.txt | 20 + .../arrow/licenses/org-immutables-LICENSE.txt | 201 +++++++ .../arrow/licenses/org-immutables-NOTICE.txt | 0 .../arrow/licenses/slf4j-LICENSE.txt | 0 libs/arrow/licenses/slf4j-NOTICE.txt | 0 libs/arrow/src/main/java/module-info.java | 20 + .../java/org/elasticsearch/arrow/Arrow.java | 33 ++ .../arrow/ArrowFormatException.java | 17 + .../elasticsearch/arrow/ArrowJsonParser.java | 242 +++++++++ .../arrow/ArrowJsonXContentParser.java | 313 +++++++++++ .../elasticsearch/arrow/ArrowToXContent.java | 271 ++++++++++ .../elasticsearch/arrow/XContentBuffer.java | 444 ++++++++++++++++ .../arrow/ArrowToXContentTests.java | 104 ++++ .../EntitlementInitialization.java | 12 + libs/x-content/impl/build.gradle | 10 +- .../impl/src/main/java/module-info.java | 2 +- modules/ingest-attachment/build.gradle | 80 +-- server/build.gradle | 9 + .../action/bulk/arrow/BulkArrowIT.java | 182 +++++++ server/src/main/java/module-info.java | 8 +- .../bulk/AbstractBulkRequestParser.java | 75 +++ .../action/bulk/BulkRequestParser.java | 493 ++++++++++-------- .../arrow/ArrowBulkIncrementalParser.java | 357 +++++++++++++ .../bulk/arrow/ArrowBulkRequestParser.java | 124 +++++ .../bulk/arrow/ArrowIncrementalParser.java | 159 ++++++ .../bulk/arrow/BytesReferenceChannel.java | 85 +++ .../elasticsearch/rest/RestController.java | 1 + .../rest/action/document/RestBulkAction.java | 19 +- .../elasticsearch/bootstrap/security.policy | 18 + .../ArrowBulkIncrementalParserTests.java | 470 +++++++++++++++++ .../arrow/BytesReferenceChannelTests.java | 50 ++ x-pack/plugin/esql/arrow/build.gradle | 41 +- .../esql/arrow/AllocationManagerShim.java | 69 --- .../xpack/esql/arrow/ArrowResponse.java | 5 - .../esql/qa/server/single-node/build.gradle | 13 +- .../plugin-metadata/plugin-security.codebases | 1 - .../plugin-metadata/plugin-security.policy | 12 - 56 files changed, 4181 insertions(+), 434 deletions(-) create mode 100644 libs/arrow/build.gradle rename {x-pack/plugin/esql => libs}/arrow/licenses/arrow-LICENSE.txt (100%) rename {x-pack/plugin/esql => libs}/arrow/licenses/arrow-NOTICE.txt (100%) create mode 100644 libs/arrow/licenses/build.gradle rename {x-pack/plugin/esql => libs}/arrow/licenses/checker-qual-LICENSE.txt (100%) rename {x-pack/plugin/esql => libs}/arrow/licenses/checker-qual-NOTICE.txt (100%) rename x-pack/plugin/esql/arrow/licenses/flatbuffers-java-LICENSE.txt => libs/arrow/licenses/commons-codec-arrow-LICENSE.txt (100%) create mode 100644 libs/arrow/licenses/commons-codec-arrow-NOTICE.txt create mode 100644 libs/arrow/licenses/error_prone_annotations-LICENSE.txt rename x-pack/plugin/esql/arrow/licenses/flatbuffers-java-NOTICE.txt => libs/arrow/licenses/error_prone_annotations-NOTICE.txt (100%) rename x-pack/plugin/esql/arrow/licenses/jackson-LICENSE.txt => libs/arrow/licenses/flatbuffers-java-LICENSE.txt (100%) rename x-pack/plugin/esql/arrow/licenses/jackson-NOTICE.txt => libs/arrow/licenses/flatbuffers-java-NOTICE.txt (100%) create mode 100644 libs/arrow/licenses/jackson-LICENSE.txt create mode 100644 libs/arrow/licenses/jackson-NOTICE.txt create mode 100644 libs/arrow/licenses/org-immutables-LICENSE.txt rename x-pack/plugin/esql/arrow/licenses/slf4j-NOTICE.txt => libs/arrow/licenses/org-immutables-NOTICE.txt (100%) rename {x-pack/plugin/esql => libs}/arrow/licenses/slf4j-LICENSE.txt (100%) create mode 100644 libs/arrow/licenses/slf4j-NOTICE.txt create mode 100644 libs/arrow/src/main/java/module-info.java create mode 100644 libs/arrow/src/main/java/org/elasticsearch/arrow/Arrow.java create mode 100644 libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowFormatException.java create mode 100644 libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonParser.java create mode 100644 libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonXContentParser.java create mode 100644 libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowToXContent.java create mode 100644 libs/arrow/src/main/java/org/elasticsearch/arrow/XContentBuffer.java create mode 100644 libs/arrow/src/test/java/org/elasticsearch/arrow/ArrowToXContentTests.java create mode 100644 server/src/javaRestTest/java/org/elasticsearch/action/bulk/arrow/BulkArrowIT.java create mode 100644 server/src/main/java/org/elasticsearch/action/bulk/AbstractBulkRequestParser.java create mode 100644 server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParser.java create mode 100644 server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkRequestParser.java create mode 100644 server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowIncrementalParser.java create mode 100644 server/src/main/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannel.java create mode 100644 server/src/test/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParserTests.java create mode 100644 server/src/test/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannelTests.java delete mode 100644 x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/AllocationManagerShim.java delete mode 100644 x-pack/plugin/esql/src/main/plugin-metadata/plugin-security.codebases delete mode 100644 x-pack/plugin/esql/src/main/plugin-metadata/plugin-security.policy diff --git a/.idea/inspectionProfiles/Project_Default.xml b/.idea/inspectionProfiles/Project_Default.xml index 34ff98c18372f..9e9a0f2dc634f 100644 --- a/.idea/inspectionProfiles/Project_Default.xml +++ b/.idea/inspectionProfiles/Project_Default.xml @@ -1,6 +1,9 @@ - + \ No newline at end of file diff --git a/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/ElasticsearchTestBasePlugin.java b/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/ElasticsearchTestBasePlugin.java index fa75a2011ab90..6bbd3d31d5030 100644 --- a/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/ElasticsearchTestBasePlugin.java +++ b/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/ElasticsearchTestBasePlugin.java @@ -120,6 +120,9 @@ public void execute(Task t) { "--add-opens=java.base/java.nio.file=ALL-UNNAMED", "--add-opens=java.base/java.time=ALL-UNNAMED", "--add-opens=java.management/java.lang.management=ALL-UNNAMED", + "--add-opens=java.base/java.nio=org.apache.arrow.memory.core,ALL-UNNAMED", + // Define the allocation manager type to avoid classpath scanning to locate one. + "-Darrow.allocation.manager.type=Unsafe", "-XX:+HeapDumpOnOutOfMemoryError" ); diff --git a/build-tools-internal/version.properties b/build-tools-internal/version.properties index a0341c9c5035f..bfee3fbea6782 100644 --- a/build-tools-internal/version.properties +++ b/build-tools-internal/version.properties @@ -11,7 +11,7 @@ snakeyaml = 2.0 icu4j = 68.2 supercsv = 2.4.0 log4j = 2.19.0 -slf4j = 2.0.6 +slf4j = 2.0.10 ecsLogging = 1.2.0 jna = 5.12.1 netty = 4.1.118.Final @@ -32,7 +32,7 @@ httpclient = 4.5.14 httpcore = 4.4.16 httpasyncclient = 4.1.5 commonslogging = 1.2 -commonscodec = 1.15 +commonscodec = 1.17.1 protobuf = 3.25.5 # test dependencies diff --git a/distribution/src/config/jvm.options b/distribution/src/config/jvm.options index 94fc6f2cb9025..786c0a81c5540 100644 --- a/distribution/src/config/jvm.options +++ b/distribution/src/config/jvm.options @@ -84,3 +84,10 @@ ## GC logging -Xlog:gc*,gc+age=trace,safepoint:file=@loggc@:utctime,level,pid,tags:filecount=32,filesize=64m + +## Arrow +# Allow accessing a private field of java.nio.Buffer for direct memory access. +# See org.apache.arrow.memory.MemoryUtil and https://arrow.apache.org/docs/java/install.html +--add-opens=java.base/java.nio=org.apache.arrow.memory.core +# Define the allocation manager type to avoid classpath scanning to locate one. +-Darrow.allocation.manager.type=Unsafe diff --git a/gradle/build.versions.toml b/gradle/build.versions.toml index 3635e26dcc21c..2964235e79241 100644 --- a/gradle/build.versions.toml +++ b/gradle/build.versions.toml @@ -1,6 +1,6 @@ [versions] asm = "9.7.1" -jackson = "2.15.0" +jackson = "2.17.2" junit5 = "5.8.1" spock = "2.1-groovy-3.0" diff --git a/gradle/verification-metadata.xml b/gradle/verification-metadata.xml index 328b696ebf67d..f62b9ea3a0455 100644 --- a/gradle/verification-metadata.xml +++ b/gradle/verification-metadata.xml @@ -294,6 +294,11 @@ + + + + + @@ -344,6 +349,11 @@ + + + + + @@ -624,11 +634,21 @@ + + + + + + + + + + @@ -1219,6 +1239,11 @@ + + + + + @@ -1935,24 +1960,24 @@ - - - + + + - - - + + + - - - + + + - - - + + + @@ -3401,6 +3426,11 @@ + + + + + @@ -3891,6 +3921,16 @@ + + + + + + + + + + @@ -4486,6 +4526,11 @@ + + + + + @@ -4546,6 +4591,11 @@ + + + + + diff --git a/libs/arrow/build.gradle b/libs/arrow/build.gradle new file mode 100644 index 0000000000000..1c9676131ad19 --- /dev/null +++ b/libs/arrow/build.gradle @@ -0,0 +1,91 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +// Notes: +// - additional JVM arguments are added to distribution/src/config/jvm.options and ElasticsearchTestBasePlugin +// - additional permissions are added to server/src/main/resources/org/elasticsearch/bootstrap/security.policy + +apply plugin: 'elasticsearch.build' +apply plugin: 'elasticsearch.publish' + +var arrowVersion = "18.2.0" + +dependencies { + implementation(project(":libs:x-content")) + + // jackson-core is provided by :libs:x-content:impl. If declared here, there's a module issue that prevents ES from starting: + // + // fatal exception while booting Elasticsearch java.lang.IllegalAccessError: class org.elasticsearch.xcontent.provider.json.JsonXContentImpl (in module org.elasticsearch.xcontent.impl) cannot access class com.fasterxml.jackson.core.JsonFactoryBuilder (in unnamed module @0x4727e5fc) because module org.elasticsearch.xcontent.impl does not read unnamed module @0x4727e5fc + // at org.elasticsearch.xcontent.impl@9.0.0-SNAPSHOT/org.elasticsearch.xcontent.provider.json.JsonXContentImpl.(JsonXContentImpl.java:50) + // at org.elasticsearch.xcontent.impl@9.0.0-SNAPSHOT/org.elasticsearch.xcontent.provider.XContentProviderImpl$2.XContent(XContentProviderImpl.java:54) + // at org.elasticsearch.xcontent@9.0.0-SNAPSHOT/org.elasticsearch.xcontent.json.JsonXContent.(JsonXContent.java:37) + // at org.elasticsearch.xcontent@9.0.0-SNAPSHOT/org.elasticsearch.xcontent.XContentType.(XContentType.java:28) + // at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.common.settings.Setting.arrayToParsableString(Setting.java:1883) + implementation(project(":libs:x-content:impl")) + + // arrow-vector + api("org.apache.arrow:arrow-vector:${arrowVersion}") + //api("com.fasterxml.jackson.core:jackson-core:${versions.jackson}") + api("com.fasterxml.jackson.core:jackson-annotations:${versions.jackson}") + api("com.fasterxml.jackson.core:jackson-databind:${versions.jackson}") + api("com.fasterxml.jackson.datatype:jackson-datatype-jsr310:${versions.jackson}") + + api("com.google.flatbuffers:flatbuffers-java:24.3.25") + api("commons-codec:commons-codec:${versions.commonscodec}") // Arrow 18 -> commons-codec 1.17.1 + api("org.slf4j:slf4j-api:${versions.slf4j}") + api("org.immutables:value-annotations:2.10.1") // provided dependency + + // arrow-format + api("org.apache.arrow:arrow-format:${arrowVersion}") + // also depends on flatbuffers + + // arrow-memory-core + api("org.apache.arrow:arrow-memory-core:${arrowVersion}") + api("com.google.errorprone:error_prone_annotations:2.31.0") // provided dependency + api('org.checkerframework:checker-qual:3.48.1') // provided dependency + // also depends on value-annotations (provided dependency) + + // arrow-memory-unsafe + implementation("org.apache.arrow:arrow-memory-unsafe:${arrowVersion}") + // also depends on value-annotations (provided dependency) + + testImplementation(project(":test:framework")) { + exclude group: 'org.elasticsearch', module: 'arrow' + } +} + +tasks.named("dependencyLicenses").configure { + mapping from: /jackson-.*/, to: 'jackson' + mapping from: /arrow-.*/, to: 'arrow' + mapping from: /slf4j-.*/, to: 'slf4j' + mapping from: /value-annotations.*/, to: 'org-immutables' + + // FIXME: need dependency above, but build complains about duplicated license files. + mapping from: /commons-codec/, to: 'commons-codec-arrow' +} + +tasks.named("thirdPartyAudit").configure { + ignoreViolations( + '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' + ) +} diff --git a/x-pack/plugin/esql/arrow/licenses/arrow-LICENSE.txt b/libs/arrow/licenses/arrow-LICENSE.txt similarity index 100% rename from x-pack/plugin/esql/arrow/licenses/arrow-LICENSE.txt rename to libs/arrow/licenses/arrow-LICENSE.txt diff --git a/x-pack/plugin/esql/arrow/licenses/arrow-NOTICE.txt b/libs/arrow/licenses/arrow-NOTICE.txt similarity index 100% rename from x-pack/plugin/esql/arrow/licenses/arrow-NOTICE.txt rename to libs/arrow/licenses/arrow-NOTICE.txt diff --git a/libs/arrow/licenses/build.gradle b/libs/arrow/licenses/build.gradle new file mode 100644 index 0000000000000..0fe358c4b3703 --- /dev/null +++ b/libs/arrow/licenses/build.gradle @@ -0,0 +1,49 @@ +/* + * 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. + */ + +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 project(":libs:arrow") + testImplementation project(':test:framework') +} + +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' + ) +} + +tasks.named("test").configure { + jvmArgs('--add-opens=java.base/java.nio=ALL-UNNAMED') +} diff --git a/x-pack/plugin/esql/arrow/licenses/checker-qual-LICENSE.txt b/libs/arrow/licenses/checker-qual-LICENSE.txt similarity index 100% rename from x-pack/plugin/esql/arrow/licenses/checker-qual-LICENSE.txt rename to libs/arrow/licenses/checker-qual-LICENSE.txt diff --git a/x-pack/plugin/esql/arrow/licenses/checker-qual-NOTICE.txt b/libs/arrow/licenses/checker-qual-NOTICE.txt similarity index 100% rename from x-pack/plugin/esql/arrow/licenses/checker-qual-NOTICE.txt rename to libs/arrow/licenses/checker-qual-NOTICE.txt diff --git a/x-pack/plugin/esql/arrow/licenses/flatbuffers-java-LICENSE.txt b/libs/arrow/licenses/commons-codec-arrow-LICENSE.txt similarity index 100% rename from x-pack/plugin/esql/arrow/licenses/flatbuffers-java-LICENSE.txt rename to libs/arrow/licenses/commons-codec-arrow-LICENSE.txt diff --git a/libs/arrow/licenses/commons-codec-arrow-NOTICE.txt b/libs/arrow/licenses/commons-codec-arrow-NOTICE.txt new file mode 100644 index 0000000000000..1da9af50f6008 --- /dev/null +++ b/libs/arrow/licenses/commons-codec-arrow-NOTICE.txt @@ -0,0 +1,17 @@ +Apache Commons Codec +Copyright 2002-2014 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java +contains test data from http://aspell.net/test/orig/batch0.tab. +Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + +=============================================================================== + +The content of package org.apache.commons.codec.language.bm has been translated +from the original php source code available at http://stevemorse.org/phoneticinfo.htm +with permission from the original authors. +Original source copyright: +Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/libs/arrow/licenses/error_prone_annotations-LICENSE.txt b/libs/arrow/licenses/error_prone_annotations-LICENSE.txt new file mode 100644 index 0000000000000..5c304d1a4a7b4 --- /dev/null +++ b/libs/arrow/licenses/error_prone_annotations-LICENSE.txt @@ -0,0 +1,201 @@ +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. diff --git a/x-pack/plugin/esql/arrow/licenses/flatbuffers-java-NOTICE.txt b/libs/arrow/licenses/error_prone_annotations-NOTICE.txt similarity index 100% rename from x-pack/plugin/esql/arrow/licenses/flatbuffers-java-NOTICE.txt rename to libs/arrow/licenses/error_prone_annotations-NOTICE.txt diff --git a/x-pack/plugin/esql/arrow/licenses/jackson-LICENSE.txt b/libs/arrow/licenses/flatbuffers-java-LICENSE.txt similarity index 100% rename from x-pack/plugin/esql/arrow/licenses/jackson-LICENSE.txt rename to libs/arrow/licenses/flatbuffers-java-LICENSE.txt diff --git a/x-pack/plugin/esql/arrow/licenses/jackson-NOTICE.txt b/libs/arrow/licenses/flatbuffers-java-NOTICE.txt similarity index 100% rename from x-pack/plugin/esql/arrow/licenses/jackson-NOTICE.txt rename to libs/arrow/licenses/flatbuffers-java-NOTICE.txt diff --git a/libs/arrow/licenses/jackson-LICENSE.txt b/libs/arrow/licenses/jackson-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/libs/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. diff --git a/libs/arrow/licenses/jackson-NOTICE.txt b/libs/arrow/licenses/jackson-NOTICE.txt new file mode 100644 index 0000000000000..4c976b7b4cc58 --- /dev/null +++ b/libs/arrow/licenses/jackson-NOTICE.txt @@ -0,0 +1,20 @@ +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. diff --git a/libs/arrow/licenses/org-immutables-LICENSE.txt b/libs/arrow/licenses/org-immutables-LICENSE.txt new file mode 100644 index 0000000000000..5c304d1a4a7b4 --- /dev/null +++ b/libs/arrow/licenses/org-immutables-LICENSE.txt @@ -0,0 +1,201 @@ +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. diff --git a/x-pack/plugin/esql/arrow/licenses/slf4j-NOTICE.txt b/libs/arrow/licenses/org-immutables-NOTICE.txt similarity index 100% rename from x-pack/plugin/esql/arrow/licenses/slf4j-NOTICE.txt rename to libs/arrow/licenses/org-immutables-NOTICE.txt diff --git a/x-pack/plugin/esql/arrow/licenses/slf4j-LICENSE.txt b/libs/arrow/licenses/slf4j-LICENSE.txt similarity index 100% rename from x-pack/plugin/esql/arrow/licenses/slf4j-LICENSE.txt rename to libs/arrow/licenses/slf4j-LICENSE.txt diff --git a/libs/arrow/licenses/slf4j-NOTICE.txt b/libs/arrow/licenses/slf4j-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/libs/arrow/src/main/java/module-info.java b/libs/arrow/src/main/java/module-info.java new file mode 100644 index 0000000000000..f6c0d3e1976dd --- /dev/null +++ b/libs/arrow/src/main/java/module-info.java @@ -0,0 +1,20 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +module org.elasticsearch.arrow { + exports org.elasticsearch.arrow; + + requires transitive org.apache.arrow.vector; + requires transitive org.apache.arrow.memory.core; + requires transitive org.apache.arrow.format; + + requires org.elasticsearch.xcontent; + requires com.fasterxml.jackson.databind; + requires org.elasticsearch.base; +} diff --git a/libs/arrow/src/main/java/org/elasticsearch/arrow/Arrow.java b/libs/arrow/src/main/java/org/elasticsearch/arrow/Arrow.java new file mode 100644 index 0000000000000..e5764da5d3e3b --- /dev/null +++ b/libs/arrow/src/main/java/org/elasticsearch/arrow/Arrow.java @@ -0,0 +1,33 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.arrow; + +import org.apache.arrow.memory.RootAllocator; + +public class Arrow { + + /** + * Arrow IPC stream media type. + * + * @see Format docs + * @see IANA assignment + */ + public static String MEDIA_TYPE = "application/vnd.apache.arrow.stream"; + + private static final RootAllocator ROOT_ALLOCATOR = new RootAllocator(); + + /** + * Returns the global root allocator. Should be used to create child allocators to have + * fine-grained memory allocation tracking and to enforce local limits. + */ + public static RootAllocator rootAllocator() { + return ROOT_ALLOCATOR; + } +} diff --git a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowFormatException.java b/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowFormatException.java new file mode 100644 index 0000000000000..c0068ff7d9578 --- /dev/null +++ b/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowFormatException.java @@ -0,0 +1,17 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.arrow; + +public class ArrowFormatException extends RuntimeException { + + public ArrowFormatException(String msg) { + super(msg); + } +} diff --git a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonParser.java b/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonParser.java new file mode 100644 index 0000000000000..557562ea3f9f8 --- /dev/null +++ b/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonParser.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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.arrow; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParseException; +import com.fasterxml.jackson.core.JsonToken; +import com.fasterxml.jackson.core.util.JsonParserDelegate; +import com.fasterxml.jackson.databind.util.TokenBuffer; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.BaseIntVector; +import org.apache.arrow.vector.BitVector; +import org.apache.arrow.vector.FloatingPointVector; +import org.apache.arrow.vector.TimeStampVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.VarBinaryVector; +import org.apache.arrow.vector.VariableWidthFieldVector; +import org.apache.arrow.vector.complex.BaseListVector; +import org.apache.arrow.vector.complex.DenseUnionVector; +import org.apache.arrow.vector.complex.MapVector; +import org.apache.arrow.vector.complex.StructVector; +import org.apache.arrow.vector.complex.UnionVector; +import org.apache.arrow.vector.ipc.ArrowStreamReader; +import org.apache.arrow.vector.types.Types; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.EnumSet; + +/** + * A Jackson parser that reads Arrow record batches. The JSON result is an array + * containing one object per line. + *

+ * To reduce memory allocations, each line is read in a Jackson TokenBuffer, created + * lazily as the parser consumer fetched events. Similarly, record batches are read + * lazily. + */ +public class ArrowJsonParser extends JsonParserDelegate { + + private static final EnumSet STRING_TYPES = EnumSet.of( + Types.MinorType.VARCHAR, + Types.MinorType.LARGEVARCHAR, + Types.MinorType.VIEWVARCHAR + ); + + private final ArrowStreamReader reader; + private boolean done = false; + + private int rootPosition = 0; + + public ArrowJsonParser(InputStream in) throws IOException { + this(in, new RootAllocator()); + } + + public ArrowJsonParser(InputStream in, BufferAllocator allocator) throws IOException { + super(null); + this.reader = new ArrowStreamReader(in, allocator); + // Read schema early + reader.getVectorSchemaRoot(); + + var p = new JsonFactory().createParser(""); + var tokens = new TokenBuffer(p); + tokens.writeStartArray(); + delegate = tokens.asParser(); + } + + @Override + public JsonToken nextToken() throws IOException { + var result = delegate.nextToken(); + if (result == null && done == false) { + fillBuffer(); + return nextToken(); + } else { + return result; + } + } + + private void fillBuffer() throws IOException { + if (done) { + return; + } + + var buffer = new TokenBuffer(this); + + if (rootPosition == 0) { + if (this.reader.loadNextBatch() == false) { + // End of stream + buffer.writeEndArray(); + delegate = buffer.asParser(); + done = true; + return; + } + } + + var schema = reader.getVectorSchemaRoot(); + + buffer.writeStartObject(); + for (var vector : schema.getFieldVectors()) { + buffer.writeFieldName(vector.getName()); + writeValue(vector, rootPosition, buffer); + } + buffer.writeEndObject(); + + rootPosition++; + if (rootPosition >= schema.getRowCount()) { + // Read a new batch at the next iteration + rootPosition = 0; + } + delegate = buffer.asParser(); + } + + private void writeValue(ValueVector vector, int position, JsonGenerator generator) throws IOException { + + if (vector.isNull(position)) { + generator.writeNull(); + return; + } + + switch (vector.getMinorType()) { + case TINYINT, SMALLINT, INT, BIGINT, UINT1, UINT2, UINT4, UINT8 -> { + generator.writeNumber(((BaseIntVector) vector).getValueAsLong(position)); + } + + case FLOAT2, FLOAT4, FLOAT8 -> { + generator.writeNumber(((FloatingPointVector) vector).getValueAsDouble(position)); + } + + case BIT -> { + generator.writeBoolean(((BitVector) vector).get(position) != 0); + } + + case VARCHAR, LARGEVARCHAR, VIEWVARCHAR -> { + var bytesVector = (VariableWidthFieldVector) vector; + // TODO: maybe we can avoid a copy using bytesVector.getDatapointer()? + generator.writeString(new String(bytesVector.get(position), StandardCharsets.UTF_8)); + } + + case VARBINARY, LARGEVARBINARY, VIEWVARBINARY -> { + var bytesVector = (VariableWidthFieldVector) vector; + generator.writeBinary(bytesVector.get(position)); + } + + case LIST, FIXED_SIZE_LIST, LISTVIEW -> { + var listVector = (BaseListVector) vector; + var valueVector = listVector.getChildrenFromFields().get(0); + int start = listVector.getElementStartIndex(position); + int end = listVector.getElementEndIndex(position); + + generator.writeStartArray(); + for (int i = start; i < end; i++) { + writeValue(valueVector, i, generator); + } + generator.writeEndArray(); + } + + case TIMESTAMPMILLI -> { + var tsVector = (TimeStampVector) vector; + generator.writeNumber(tsVector.get(position)); + } + + case TIMEMICRO -> { + var tsVector = (TimeStampVector) vector; + // FIXME: format as string with enough decimal positions + generator.writeNumber(tsVector.get(position) / 1000); + } + + case TIMENANO -> { + var tsVector = (TimeStampVector) vector; + // FIXME: format as string with enough decimal positions + generator.writeNumber(tsVector.get(position) / 1_000_000); + } + + case MAP -> { + // A map is a container vector that is composed of a list of struct values with "key" and "value" fields. The MapVector + // is nullable, but if a map is set at a given index, there must be an entry. In other words, the StructVector data is + // non-nullable. Also for a given entry, the "key" is non-nullable, however the "value" can be null. + + var mapVector = (MapVector) vector; + var structVector = (StructVector) mapVector.getChildrenFromFields().get(0); + var kVector = structVector.getChildrenFromFields().get(0); + if (STRING_TYPES.contains(kVector.getMinorType()) == false) { + throw new ArrowFormatException("Maps must have string keys"); + } + + var keyVector = (VarBinaryVector) kVector; + var valueVector = structVector.getChildrenFromFields().get(1); + + int start = mapVector.getElementStartIndex(position); + int end = mapVector.getElementEndIndex(position); + + generator.writeStartObject(); + for (int i = start; i < end; i++) { + var key = new String(keyVector.get(i), StandardCharsets.UTF_8); + generator.writeFieldName(key); + writeValue(valueVector, i, generator); + } + generator.writeEndObject(); + } + + case STRUCT -> { + var structVector = (StructVector) vector; + generator.writeStartObject(); + for (var field : structVector.getChildrenFromFields()) { + generator.writeFieldName(field.getName()); + writeValue(field, position, generator); + } + generator.writeEndObject(); + } + + case DENSEUNION -> { + var unionVector = (DenseUnionVector) vector; + var typeId = unionVector.getTypeId(position); + var valueVector = unionVector.getVectorByType(typeId); + var valuePosition = unionVector.getOffset(position); + + writeValue(valueVector, valuePosition, generator); + } + + case UNION -> { // sparse union + var unionVector = (UnionVector) vector; + var typeId = unionVector.getTypeValue(position); + var valueVector = unionVector.getVectorByType(typeId); + + writeValue(valueVector, position, generator); + } + + default -> throw new JsonParseException( + "Arrow type [" + vector.getMinorType() + "] not supported for field [" + vector.getName() + "]" + ); + } + } +} diff --git a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonXContentParser.java b/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonXContentParser.java new file mode 100644 index 0000000000000..f5cb63ef99292 --- /dev/null +++ b/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonXContentParser.java @@ -0,0 +1,313 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.arrow; + +import com.fasterxml.jackson.core.JsonLocation; +import com.fasterxml.jackson.core.JsonParseException; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.JsonToken; +import com.fasterxml.jackson.core.exc.InputCoercionException; +import com.fasterxml.jackson.core.io.JsonEOFException; + +import org.elasticsearch.xcontent.XContentEOFException; +import org.elasticsearch.xcontent.XContentLocation; +import org.elasticsearch.xcontent.XContentParseException; +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xcontent.support.AbstractXContentParser; + +import java.io.IOException; +import java.nio.CharBuffer; + +// copy of JsonXContentParser in :libs:x-content:impl +class ArrowJsonXContentParser extends AbstractXContentParser { + + final JsonParser parser; + + ArrowJsonXContentParser(XContentParserConfiguration config, JsonParser parser) { + super(config.registry(), config.deprecationHandler(), config.restApiVersion()); + //this.parser = ((XContentParserConfigurationImpl) config).filter(parser); + this.parser = parser; + } + + @Override + public XContentType contentType() { + return XContentType.JSON; + } + + @Override + public void allowDuplicateKeys(boolean allowDuplicateKeys) { + parser.configure(JsonParser.Feature.STRICT_DUPLICATE_DETECTION, allowDuplicateKeys == false); + } + + private static XContentParseException newXContentParseException(JsonProcessingException e) { + JsonLocation loc = e.getLocation(); + throw new XContentParseException(new XContentLocation(loc.getLineNr(), loc.getColumnNr()), e.getMessage(), e); + } + + @Override + public Token nextToken() throws IOException { + try { + return convertToken(parser.nextToken()); + } catch (JsonEOFException e) { + JsonLocation location = e.getLocation(); + throw new XContentEOFException(new XContentLocation(location.getLineNr(), location.getColumnNr()), "Unexpected end of file", e); + } catch (JsonParseException e) { + throw newXContentParseException(e); + } + } + + @Override + public String nextFieldName() throws IOException { + try { + return parser.nextFieldName(); + } catch (JsonParseException e) { + throw newXContentParseException(e); + } + } + + @Override + public void skipChildren() throws IOException { + parser.skipChildren(); + } + + @Override + public Token currentToken() { + return convertToken(parser.getCurrentToken()); + } + + @Override + public NumberType numberType() throws IOException { + return convertNumberType(parser.getNumberType()); + } + + @Override + public String currentName() throws IOException { + return parser.getCurrentName(); + } + + @Override + protected boolean doBooleanValue() throws IOException { + try { + return parser.getBooleanValue(); + } catch (JsonParseException e) { + throw newXContentParseException(e); + } + } + + @Override + public String text() throws IOException { + if (currentToken().isValue() == false) { + throwOnNoText(); + } + return parser.getText(); + } + + private void throwOnNoText() { + throw new IllegalArgumentException("Expected text at " + getTokenLocation() + " but found " + currentToken()); + } + + @Override + public CharBuffer charBuffer() throws IOException { + try { + return CharBuffer.wrap(parser.getTextCharacters(), parser.getTextOffset(), parser.getTextLength()); + } catch (JsonParseException e) { + throw newXContentParseException(e); + } + } + + @Override + public Object objectText() throws IOException { + JsonToken currentToken = parser.getCurrentToken(); + if (currentToken == JsonToken.VALUE_STRING) { + return text(); + } else if (currentToken == JsonToken.VALUE_NUMBER_INT || currentToken == JsonToken.VALUE_NUMBER_FLOAT) { + return parser.getNumberValue(); + } else if (currentToken == JsonToken.VALUE_TRUE) { + return Boolean.TRUE; + } else if (currentToken == JsonToken.VALUE_FALSE) { + return Boolean.FALSE; + } else if (currentToken == JsonToken.VALUE_NULL) { + return null; + } else { + return text(); + } + } + + @Override + public Object objectBytes() throws IOException { + JsonToken currentToken = parser.getCurrentToken(); + if (currentToken == JsonToken.VALUE_STRING) { + return charBuffer(); + } else if (currentToken == JsonToken.VALUE_NUMBER_INT || currentToken == JsonToken.VALUE_NUMBER_FLOAT) { + return parser.getNumberValue(); + } else if (currentToken == JsonToken.VALUE_TRUE) { + return Boolean.TRUE; + } else if (currentToken == JsonToken.VALUE_FALSE) { + return Boolean.FALSE; + } else if (currentToken == JsonToken.VALUE_NULL) { + return null; + } else { + return charBuffer(); + } + } + + @Override + public boolean hasTextCharacters() { + return parser.hasTextCharacters(); + } + + @Override + public char[] textCharacters() throws IOException { + try { + return parser.getTextCharacters(); + } catch (JsonParseException e) { + throw newXContentParseException(e); + } + } + + @Override + public int textLength() throws IOException { + try { + return parser.getTextLength(); + } catch (JsonParseException e) { + throw newXContentParseException(e); + } + } + + @Override + public int textOffset() throws IOException { + try { + return parser.getTextOffset(); + } catch (JsonParseException e) { + throw newXContentParseException(e); + } + } + + @Override + public Number numberValue() throws IOException { + try { + return parser.getNumberValue(); + } catch (InputCoercionException | JsonParseException e) { + throw newXContentParseException(e); + } + } + + @Override + public short doShortValue() throws IOException { + try { + return parser.getShortValue(); + } catch (InputCoercionException | JsonParseException e) { + throw newXContentParseException(e); + } + } + + @Override + public int doIntValue() throws IOException { + try { + return parser.getIntValue(); + } catch (InputCoercionException | JsonParseException e) { + throw newXContentParseException(e); + } + } + + @Override + public long doLongValue() throws IOException { + try { + return parser.getLongValue(); + } catch (InputCoercionException | JsonParseException e) { + throw newXContentParseException(e); + } + } + + @Override + public float doFloatValue() throws IOException { + try { + return parser.getFloatValue(); + } catch (InputCoercionException | JsonParseException e) { + throw newXContentParseException(e); + } + } + + @Override + public double doDoubleValue() throws IOException { + try { + return parser.getDoubleValue(); + } catch (InputCoercionException | JsonParseException e) { + throw newXContentParseException(e); + } + } + + @Override + public byte[] binaryValue() throws IOException { + try { + return parser.getBinaryValue(); + } catch (JsonParseException e) { + throw newXContentParseException(e); + } + } + + @Override + public XContentLocation getTokenLocation() { + JsonLocation loc = parser.getTokenLocation(); + if (loc == null) { + return null; + } + return new XContentLocation(loc.getLineNr(), loc.getColumnNr()); + } + + @Override + public void close() { + // noinspection EmptyCatchBlock + try { + parser.close(); + } catch (final IOException | RuntimeException e) {} + } + + private static NumberType convertNumberType(JsonParser.NumberType numberType) { + return switch (numberType) { + case INT -> NumberType.INT; + case BIG_INTEGER -> NumberType.BIG_INTEGER; + case LONG -> NumberType.LONG; + case FLOAT -> NumberType.FLOAT; + case DOUBLE -> NumberType.DOUBLE; + case BIG_DECIMAL -> NumberType.BIG_DECIMAL; + }; + } + + private static Token convertToken(JsonToken token) { + if (token == null) { + return null; + } + return switch (token) { + case START_OBJECT -> Token.START_OBJECT; + case END_OBJECT -> Token.END_OBJECT; + case START_ARRAY -> Token.START_ARRAY; + case END_ARRAY -> Token.END_ARRAY; + case FIELD_NAME -> Token.FIELD_NAME; + case VALUE_EMBEDDED_OBJECT -> Token.VALUE_EMBEDDED_OBJECT; + case VALUE_STRING -> Token.VALUE_STRING; + case VALUE_NUMBER_INT, VALUE_NUMBER_FLOAT -> Token.VALUE_NUMBER; + case VALUE_FALSE, VALUE_TRUE -> Token.VALUE_BOOLEAN; + case VALUE_NULL -> Token.VALUE_NULL; + default -> throw unknownTokenException(token); + }; + } + + private static IllegalStateException unknownTokenException(JsonToken token) { + return new IllegalStateException("No matching token for json_token [" + token + "]"); + } + + @Override + public boolean isClosed() { + return parser.isClosed(); + } +} diff --git a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowToXContent.java b/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowToXContent.java new file mode 100644 index 0000000000000..7787fc8583e66 --- /dev/null +++ b/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowToXContent.java @@ -0,0 +1,271 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.arrow; + +import org.apache.arrow.vector.BaseIntVector; +import org.apache.arrow.vector.BitVector; +import org.apache.arrow.vector.FixedSizeBinaryVector; +import org.apache.arrow.vector.FloatingPointVector; +import org.apache.arrow.vector.TimeStampVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VariableWidthFieldVector; +import org.apache.arrow.vector.complex.BaseListVector; +import org.apache.arrow.vector.complex.DenseUnionVector; +import org.apache.arrow.vector.complex.MapVector; +import org.apache.arrow.vector.complex.StructVector; +import org.apache.arrow.vector.complex.UnionVector; +import org.apache.arrow.vector.dictionary.Dictionary; +import org.apache.arrow.vector.types.Types; +import org.elasticsearch.xcontent.XContentGenerator; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.EnumSet; +import java.util.Map; + +/** + * Utility methods to serialize Arrow dataframes to XContent events. + *

+ * Limitations: + *

    + *
  • time and timestamps are converted to milliseconds (no support for nanoseconds) + *
  • + *
  • some types aren't implemented + *
  • + *
+ * + * @see Arrow data types + */ +public class ArrowToXContent { + + private static final EnumSet STRING_TYPES = EnumSet.of( + Types.MinorType.VARCHAR, + Types.MinorType.LARGEVARCHAR, + Types.MinorType.VIEWVARCHAR + ); + + /** + * Write a field and its value from an Arrow vector as XContent + * + * @param vector the Arrow vector + * @param position the value position in the vector + * @param dictionaries to look up values for dictionary-encoded vectors + * @param generator XContent output + */ + public static void writeField( + ValueVector vector, int position, Map dictionaries, XContentGenerator generator + ) throws IOException { + generator.writeFieldName(vector.getName()); + writeValue(vector, position, dictionaries, generator); + } + + /** + * Write a value from an Arrow vector as XContent + * + * @param vector the Arrow vector + * @param position the value position in the vector + * @param dictionaries to look up values for dictionary-encoded vectors + * @param generator XContent output + */ + public static void writeValue( + ValueVector vector, int position, Map dictionaries, XContentGenerator generator + ) throws IOException { + if (vector.isNull(position)) { + generator.writeNull(); + return; + } + + var dictEncoding = vector.getField().getDictionary(); + if (dictEncoding != null) { + // Note: to improve performance and reduce GC thrashing, we could eagerly convert dictionary + // VarCharVectors to String arrays (likely the most frequent use of dictionaries) + Dictionary dictionary = dictionaries.get(dictEncoding.getId()); + position = (int) ((BaseIntVector)vector).getValueAsLong(position); + vector = dictionary.getVector(); + } + + Void x = switch (vector.getMinorType()) { + + //----- Primitive values + + case BIT -> { + generator.writeBoolean(((BitVector)vector).get(position) != 0); + yield null; + } + + case TINYINT, SMALLINT, INT, BIGINT, UINT1, UINT2, UINT4, UINT8 -> { + generator.writeNumber(((BaseIntVector)vector).getValueAsLong(position)); + yield null; + } + + case FLOAT2, FLOAT4, FLOAT8 -> { + generator.writeNumber(((FloatingPointVector)vector).getValueAsDouble(position)); + yield null; + } + + //----- strings and bytes + + case VARCHAR, LARGEVARCHAR, VIEWVARCHAR -> { + var bytesVector = (VariableWidthFieldVector)vector; + generator.writeString(new String(bytesVector.get(position), StandardCharsets.UTF_8)); + yield null; + } + + case VARBINARY, LARGEVARBINARY, VIEWVARBINARY -> { + var bytesVector = (VariableWidthFieldVector)vector; + generator.writeBinary(bytesVector.get(position)); + yield null; + } + + case FIXEDSIZEBINARY -> { + var bytesVector = (FixedSizeBinaryVector)vector; + generator.writeBinary(bytesVector.get(position)); + yield null; + } + + //----- lists + + case LIST, FIXED_SIZE_LIST, LISTVIEW -> { + var listVector = (BaseListVector)vector; + var valueVector = listVector.getChildrenFromFields().get(0); + int start = listVector.getElementStartIndex(position); + int end = listVector.getElementEndIndex(position); + + generator.writeStartArray(); + for (int i = start; i < end; i++) { + writeValue(valueVector, i, dictionaries, generator); + } + generator.writeEndArray(); + yield null; + } + + //----- Time & Timestamp (time + timezone) + + // Timestamps are the elapsed time since the Epoch, with an optional timezone that + // can be used for timezome-aware operations or display. Since ES date fields + // don't support timezones, we ignore it. + // See https://github.com/apache/arrow/blob/main/format/Schema.fbs + // and https://www.elastic.co/guide/en/elasticsearch/reference/current/date.html + + case TIMESEC, TIMESTAMPSEC -> { + var tsVector = (TimeStampVector)vector; + generator.writeNumber(tsVector.get(position)*1000); + yield null; + } + + case TIMEMILLI, TIMESTAMPMILLI -> { + var tsVector = (TimeStampVector)vector; + generator.writeNumber(tsVector.get(position)); + yield null; + } + + case TIMEMICRO, TIMESTAMPMICRO -> { + var tsVector = (TimeStampVector)vector; + generator.writeNumber(tsVector.get(position)/1000); + yield null; + } + + case TIMENANO, TIMESTAMPNANO -> { + var tsVector = (TimeStampVector)vector; + generator.writeNumber(tsVector.get(position)/1_000_000); + yield null; + } + + //----- Composite types + + case MAP -> { + // A map is a container vector composed of a list of struct values with "key" and "value" fields. The MapVector + // is nullable, but if a map is set at a given index, there must be an entry. In other words, the StructVector data is + // non-nullable. Also for a given entry, the "key" is non-nullable, however the "value" can be null. + + var mapVector = (MapVector)vector; + var structVector = (StructVector)mapVector.getChildrenFromFields().get(0); + var kVector = structVector.getChildrenFromFields().get(0); + if (STRING_TYPES.contains(kVector.getMinorType()) == false) { + throw new ArrowFormatException("Arrow maps must have string keys to be converted to JSON"); + } + + var keyVector = (VarCharVector)kVector; + var valueVector = structVector.getChildrenFromFields().get(1); + + int start = mapVector.getElementStartIndex(position); + int end = mapVector.getElementEndIndex(position); + + generator.writeStartObject(); + for (int i = start; i < end; i++) { + if (keyVector.isNull(i)) { + throw new ArrowFormatException("Null map key found at position [" + position + "]"); + } + var key = new String(keyVector.get(i), StandardCharsets.UTF_8); + generator.writeFieldName(key); + writeValue(valueVector, i, dictionaries, generator); + } + generator.writeEndObject(); + yield null; + } + + case STRUCT -> { + var structVector = (StructVector)vector; + generator.writeStartObject(); + for (var field: structVector.getChildrenFromFields()) { + generator.writeFieldName(field.getName()); + writeValue(field, position, dictionaries, generator); + } + generator.writeEndObject(); + yield null; + } + + case DENSEUNION -> { + var unionVector = (DenseUnionVector)vector; + var typeId = unionVector.getTypeId(position); + var valueVector = unionVector.getVectorByType(typeId); + var valuePosition = unionVector.getOffset(position); + + writeValue(valueVector, valuePosition, dictionaries, generator); + yield null; + } + + case UNION -> { // sparse union + var unionVector = (UnionVector)vector; + var typeId = unionVector.getTypeValue(position); + var valueVector = unionVector.getVectorByType(typeId); + + writeValue(valueVector, position, dictionaries, generator); + yield null; + } + + case NULL -> { + generator.writeNull(); + yield null; + } + + // TODO + case DATEDAY, + DATEMILLI, + INTERVALDAY, + INTERVALMONTHDAYNANO, + DURATION, + INTERVALYEAR, + DECIMAL, + DECIMAL256, + LARGELIST, + LARGELISTVIEW, + TIMESTAMPSECTZ, + TIMESTAMPMILLITZ, + TIMESTAMPMICROTZ, + TIMESTAMPNANOTZ, + EXTENSIONTYPE, + RUNENDENCODED -> throw new ArrowFormatException( + "Arrow type [" + vector.getMinorType() + "] not supported for field [" + vector.getName() + "]" + ); + }; + } +} diff --git a/libs/arrow/src/main/java/org/elasticsearch/arrow/XContentBuffer.java b/libs/arrow/src/main/java/org/elasticsearch/arrow/XContentBuffer.java new file mode 100644 index 0000000000000..2970c369c6505 --- /dev/null +++ b/libs/arrow/src/main/java/org/elasticsearch/arrow/XContentBuffer.java @@ -0,0 +1,444 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.arrow; + +import com.fasterxml.jackson.core.JsonGenerationException; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.util.TokenBuffer; + +import org.elasticsearch.core.CheckedConsumer; +import org.elasticsearch.xcontent.XContentFactory; +import org.elasticsearch.xcontent.XContentGenerationException; +import org.elasticsearch.xcontent.XContentGenerator; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xcontent.XContentType; + +import java.io.BufferedInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.math.BigDecimal; +import java.math.BigInteger; + +/** + * A buffer of {@code XContent} events that can be replayed as an {@code XContentParser}. Useful to create synthetic + * JSON documents that are fed to existing JSON parsers. + */ +public class XContentBuffer implements XContentGenerator { + + /** Buffer used to write content **/ + private final TokenBuffer generator; + + public XContentBuffer() { + this.generator = new TokenBuffer(new ObjectMapper(), false); + } + + /** + * Return this buffer as an {@code XContent} parser. Events can be added to the buffer while events are + * consumed from the parser, but these appends are not thread-safe. + */ + public XContentParser asParser() { + return new ArrowJsonXContentParser(XContentParserConfiguration.EMPTY, this.generator.asParser()); + } + + @Override + public XContentType contentType() { + return null; + } + + @Override + public final void usePrettyPrint() { + } + + @Override + public boolean isPrettyPrint() { + return false; + } + + @Override + public void usePrintLineFeedAtEnd() { + } + + @Override + public void writeStartObject() throws IOException { + try { + generator.writeStartObject(); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeEndObject() throws IOException { + try { + generator.writeEndObject(); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeStartArray() throws IOException { + try { + generator.writeStartArray(); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeEndArray() throws IOException { + try { + generator.writeEndArray(); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeFieldName(String name) throws IOException { + try { + generator.writeFieldName(name); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeNull() throws IOException { + try { + generator.writeNull(); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeNullField(String name) throws IOException { + try { + generator.writeNullField(name); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeBooleanField(String name, boolean value) throws IOException { + try { + generator.writeBooleanField(name, value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeBoolean(boolean value) throws IOException { + try { + generator.writeBoolean(value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeNumberField(String name, double value) throws IOException { + try { + generator.writeNumberField(name, value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeNumber(double value) throws IOException { + try { + generator.writeNumber(value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeNumberField(String name, float value) throws IOException { + try { + generator.writeNumberField(name, value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeNumber(float value) throws IOException { + try { + generator.writeNumber(value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeNumberField(String name, int value) throws IOException { + try { + generator.writeNumberField(name, value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeNumberField(String name, BigInteger value) throws IOException { + // as jackson's JsonGenerator doesn't have this method for BigInteger + // we have to implement it ourselves + try { + generator.writeFieldName(name); + generator.writeNumber(value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeNumberField(String name, BigDecimal value) throws IOException { + try { + generator.writeNumberField(name, value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeNumber(int value) throws IOException { + try { + generator.writeNumber(value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeNumberField(String name, long value) throws IOException { + try { + generator.writeNumberField(name, value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeNumber(long value) throws IOException { + try { + generator.writeNumber(value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeNumber(short value) throws IOException { + try { + generator.writeNumber(value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeNumber(BigInteger value) throws IOException { + try { + generator.writeNumber(value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeNumber(BigDecimal value) throws IOException { + try { + generator.writeNumber(value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeStringField(String name, String value) throws IOException { + try { + generator.writeStringField(name, value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeString(String value) throws IOException { + try { + generator.writeString(value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeStringArray(String[] array) throws IOException { + try { + generator.writeArray(array, 0, array.length); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeString(char[] value, int offset, int len) throws IOException { + try { + generator.writeString(value, offset, len); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeUTF8String(byte[] value, int offset, int length) throws IOException { + try { + generator.writeUTF8String(value, offset, length); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeBinaryField(String name, byte[] value) throws IOException { + try { + generator.writeBinaryField(name, value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeBinary(byte[] value) throws IOException { + try { + generator.writeBinary(value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeBinary(byte[] value, int offset, int len) throws IOException { + try { + generator.writeBinary(value, offset, len); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void writeRawField(String name, InputStream content) throws IOException { + if (content.markSupported() == false) { + // needed for the XContentFactory.xContentType call + content = new BufferedInputStream(content); + } + XContentType contentType = XContentFactory.xContentType(content); + if (contentType == null) { + throw new IllegalArgumentException("Can't write raw bytes whose xcontent-type can't be guessed"); + } + writeRawField(name, content, contentType); + } + + @Override + public void writeRawField(String name, InputStream content, XContentType contentType) throws IOException { + writeFieldName(name); + writeRawValue(content, contentType); + } + + @Override + public void writeRawValue(InputStream content, XContentType contentType) throws IOException { + try (XContentParser parser = XContentFactory.xContent(contentType).createParser(XContentParserConfiguration.EMPTY, content)) { + parser.nextToken(); + copyCurrentStructure(parser); + } + } + + @Override + public void writeRawValue(String value) throws IOException { + try { + generator.writeRawValue(value); + } catch (JsonGenerationException e) { + throw new XContentGenerationException(e); + } + } + + @Override + public void copyCurrentStructure(XContentParser parser) throws IOException { + // the start of the parser + if (parser.currentToken() == null) { + parser.nextToken(); + } + copyCurrentStructure(this, parser); + } + + /** + * Low level implementation detail of {@link XContentGenerator#copyCurrentStructure(XContentParser)}. + */ + private static void copyCurrentStructure(XContentGenerator destination, XContentParser parser) throws IOException { + XContentParser.Token token = parser.currentToken(); + + // Let's handle field-name separately first + if (token == XContentParser.Token.FIELD_NAME) { + destination.writeFieldName(parser.currentName()); + token = parser.nextToken(); + // fall-through to copy the associated value + } + + switch (token) { + case START_ARRAY -> { + destination.writeStartArray(); + while (parser.nextToken() != XContentParser.Token.END_ARRAY) { + copyCurrentStructure(destination, parser); + } + destination.writeEndArray(); + } + case START_OBJECT -> { + destination.writeStartObject(); + while (parser.nextToken() != XContentParser.Token.END_OBJECT) { + copyCurrentStructure(destination, parser); + } + destination.writeEndObject(); + } + default -> // others are simple: + destination.copyCurrentEvent(parser); + } + } + + @Override + public void writeDirectField(String name, CheckedConsumer writer) throws IOException { + throw new UnsupportedOperationException("writeDirectField is not supported"); + } + + @Override + public void flush() throws IOException { + generator.flush(); + } + + @Override + public void close() throws IOException { + if (generator.isClosed()) { + return; + } + generator.close(); + } + + @Override + public boolean isClosed() { + return generator.isClosed(); + } +} diff --git a/libs/arrow/src/test/java/org/elasticsearch/arrow/ArrowToXContentTests.java b/libs/arrow/src/test/java/org/elasticsearch/arrow/ArrowToXContentTests.java new file mode 100644 index 0000000000000..352684dc7240d --- /dev/null +++ b/libs/arrow/src/test/java/org/elasticsearch/arrow/ArrowToXContentTests.java @@ -0,0 +1,104 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.arrow; + +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.NullVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.complex.MapVector; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xcontent.XContentType; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +public class ArrowToXContentTests extends ESTestCase { + + private static void checkPosition(ValueVector vector, int position, String json) throws IOException { + var out = new ByteArrayOutputStream(); + try ( + var generator = XContentType.JSON.xContent().createGenerator(out) + ) { + generator.writeStartObject(); + ArrowToXContent.writeField(vector, position, null, generator); + generator.writeEndObject(); + } + + assertEquals(json, out.toString(StandardCharsets.UTF_8)); + } + + public void testWriteField() throws IOException { + + try ( + var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); + IntVector vector = new IntVector("intField", allocator); + ) { + vector.allocateNew(1); + vector.set(0, 123); + vector.setValueCount(1); + + checkPosition(vector, 0, "{\"intField\":123}"); + } + } + + public void testWriteVarChar() throws Exception { + try ( + var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); + VarCharVector vector = new VarCharVector("stringField", allocator); + ) { + vector.allocateNew(); + vector.set(0, "test".getBytes(StandardCharsets.UTF_8)); + vector.setValueCount(1); + + checkPosition(vector, 0, "{\"stringField\":\"test\"}"); + } + } + + public void testWriteMap() throws Exception { + try ( + var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); + MapVector vector = MapVector.empty("mapField", allocator, false); + ) { + var w = vector.getWriter(); + + w.startMap(); + w.startEntry(); + w.key().varChar().writeVarChar("key1"); + w.value().integer().writeInt(42); + w.endEntry(); + w.endMap(); + + checkPosition(vector, 0, "{\"mapField\":{\"key1\":42}}"); + } + } + + public void testWriteNullValue() throws Exception { + try ( + var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); + IntVector vector = new IntVector("intField", allocator); + ) { + vector.allocateNew(1); + vector.setNull(0); + vector.setValueCount(1); + + checkPosition(vector, 0, "{\"intField\":null}"); + } + } + + public void testWriteNullVector() throws Exception { + try ( + NullVector vector = new NullVector("nullField", 1); + ) { + checkPosition(vector, 0, "{\"nullField\":null}"); + } + } +} diff --git a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java index 682bdad3d7c0b..984e628c84b1c 100644 --- a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java +++ b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java @@ -270,6 +270,18 @@ private static PolicyManager createPolicyManager() { ); } + // In s3-repository, aws-java-sdk-core reads its default configuration from a json file in its jar. + // Its URL is opened by Jackson, causing an entitlement failure in `(server)`. + Collections.addAll( + serverScopes, + new Scope( + "com.fasterxml.jackson.core", + List.of( + new FilesEntitlement(List.of(FileData.ofPath(bootstrapArgs.modulesDir(), READ))) + ) + ) + ); + var serverPolicy = new Policy( "server", bootstrapArgs.serverPolicyPatch() == null diff --git a/libs/x-content/impl/build.gradle b/libs/x-content/impl/build.gradle index 35e122d336c68..06928662170d0 100644 --- a/libs/x-content/impl/build.gradle +++ b/libs/x-content/impl/build.gradle @@ -13,15 +13,13 @@ base { archivesName = "x-content-impl" } -String jacksonVersion = "2.17.2" - dependencies { compileOnly project(':libs:core') compileOnly project(':libs:x-content') - implementation "com.fasterxml.jackson.core:jackson-core:${jacksonVersion}" - implementation "com.fasterxml.jackson.dataformat:jackson-dataformat-smile:${jacksonVersion}" - implementation "com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:${jacksonVersion}" - implementation "com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:${jacksonVersion}" + api "com.fasterxml.jackson.core:jackson-core:${versions.jackson}" + implementation "com.fasterxml.jackson.dataformat:jackson-dataformat-smile:${versions.jackson}" + implementation "com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:${versions.jackson}" + implementation "com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:${versions.jackson}" implementation "org.yaml:snakeyaml:${versions.snakeyaml}" testImplementation(project(":test:framework")) { diff --git a/libs/x-content/impl/src/main/java/module-info.java b/libs/x-content/impl/src/main/java/module-info.java index 61ca74e4effd2..cb0f285e09ddb 100644 --- a/libs/x-content/impl/src/main/java/module-info.java +++ b/libs/x-content/impl/src/main/java/module-info.java @@ -8,7 +8,7 @@ */ module org.elasticsearch.xcontent.impl { - requires com.fasterxml.jackson.core; + requires transitive com.fasterxml.jackson.core; requires com.fasterxml.jackson.dataformat.cbor; requires com.fasterxml.jackson.dataformat.smile; requires com.fasterxml.jackson.dataformat.yaml; diff --git a/modules/ingest-attachment/build.gradle b/modules/ingest-attachment/build.gradle index a172112948fd3..388b11cef0f15 100644 --- a/modules/ingest-attachment/build.gradle +++ b/modules/ingest-attachment/build.gradle @@ -18,14 +18,14 @@ esplugin { // as we (and tika) demand, and are not interested in, say, having the same version of commons-codec as elasticsearch itself // when updating tika, please review it's parent pom : https://repo1.maven.org/maven2/org/apache/tika/tika-parent // and manually update the transitive dependencies here -def versions = [ +def buildVersions = [ 'tika' : '2.9.2', 'pdfbox': '2.0.31', 'poi' : '5.2.5', 'sparsebitset' : '1.3', //poi dependency: https://repo1.maven.org/maven2/org/apache/poi/poi/ 'mime4j': '0.8.11', - 'commonsCodec': '1.16.1', - 'slf4' : '2.0.10', + 'commonsCodec': versions.commonscodec, + 'slf4' : versions.slf4j, 'xz' : '1.9', 'commonsIo' : '2.15.1', //intentionally using the elder "juniversalchardet:juniversalchardet" rather than the newer "com.github.albfernandez:juniversalchardet" @@ -44,18 +44,18 @@ def versions = [ configurations.testRuntimeClasspath { exclude module: 'commons-logging' // The version used by POI potentially conflicts with the one pulled in by :test:framework: - resolutionStrategy.force "commons-codec:commons-codec:${versions.commonsCodec}" + resolutionStrategy.force "commons-codec:commons-codec:${buildVersions.commonsCodec}" } configurations.testCompileClasspath { // The version used by POI potentially conflicts with the one pulled in by :test:framework: - resolutionStrategy.force "commons-codec:commons-codec:${versions.commonsCodec}" + resolutionStrategy.force "commons-codec:commons-codec:${buildVersions.commonsCodec}" } dependencies { // take over logging for all dependencies - api "org.slf4j:slf4j-api:${versions.slf4}" - api "org.slf4j:jcl-over-slf4j:${versions.slf4}" + api "org.slf4j:slf4j-api:${buildVersions.slf4}" + api "org.slf4j:jcl-over-slf4j:${buildVersions.slf4}" // route slf4j over log4j // TODO blocked on https://github.com/elastic/elasticsearch/issues/93714 @@ -63,54 +63,54 @@ dependencies { // nop all slf4j logging // workaround for https://github.com/elastic/elasticsearch/issues/93714 - api "org.slf4j:slf4j-nop:${versions.slf4}" + api "org.slf4j:slf4j-nop:${buildVersions.slf4}" // mandatory for tika - api "org.apache.tika:tika-core:${versions.tika}" - api "org.apache.tika:tika-langdetect:${versions.tika}" - api "org.apache.tika:tika-langdetect-tika:${versions.tika}" - api "org.apache.tika:tika-parser-html-module:${versions.tika}" - api "org.apache.tika:tika-parser-microsoft-module:${versions.tika}" - api "org.apache.tika:tika-parser-pdf-module:${versions.tika}" - api "org.apache.tika:tika-parser-xml-module:${versions.tika}" - api "org.apache.tika:tika-parser-text-module:${versions.tika}" - api "org.apache.tika:tika-parser-miscoffice-module:${versions.tika}" - api "org.apache.tika:tika-parser-apple-module:${versions.tika}" - api "org.apache.tika:tika-parser-xmp-commons:${versions.tika}" - api "org.apache.tika:tika-parser-zip-commons:${versions.tika}" - api "org.tukaani:xz:${versions.xz}" - api "commons-io:commons-io:${versions.commonsIo}" + api "org.apache.tika:tika-core:${buildVersions.tika}" + api "org.apache.tika:tika-langdetect:${buildVersions.tika}" + api "org.apache.tika:tika-langdetect-tika:${buildVersions.tika}" + api "org.apache.tika:tika-parser-html-module:${buildVersions.tika}" + api "org.apache.tika:tika-parser-microsoft-module:${buildVersions.tika}" + api "org.apache.tika:tika-parser-pdf-module:${buildVersions.tika}" + api "org.apache.tika:tika-parser-xml-module:${buildVersions.tika}" + api "org.apache.tika:tika-parser-text-module:${buildVersions.tika}" + api "org.apache.tika:tika-parser-miscoffice-module:${buildVersions.tika}" + api "org.apache.tika:tika-parser-apple-module:${buildVersions.tika}" + api "org.apache.tika:tika-parser-xmp-commons:${buildVersions.tika}" + api "org.apache.tika:tika-parser-zip-commons:${buildVersions.tika}" + api "org.tukaani:xz:${buildVersions.xz}" + api "commons-io:commons-io:${buildVersions.commonsIo}" // character set detection - api "com.googlecode.juniversalchardet:juniversalchardet:${versions.juniversalchardet}" + api "com.googlecode.juniversalchardet:juniversalchardet:${buildVersions.juniversalchardet}" // external parser libraries // HTML - api "org.ccil.cowan.tagsoup:tagsoup:${versions.tagsoup}" + api "org.ccil.cowan.tagsoup:tagsoup:${buildVersions.tagsoup}" // Adobe PDF - api "org.apache.pdfbox:pdfbox:${versions.pdfbox}" - api "org.apache.pdfbox:fontbox:${versions.pdfbox}" - api "org.apache.pdfbox:jempbox:${versions.jempbox}" + api "org.apache.pdfbox:pdfbox:${buildVersions.pdfbox}" + api "org.apache.pdfbox:fontbox:${buildVersions.pdfbox}" + api "org.apache.pdfbox:jempbox:${buildVersions.jempbox}" // OpenOffice - api "org.apache.poi:poi-ooxml:${versions.poi}" - api "org.apache.poi:poi-ooxml-lite:${versions.poi}" - api "org.apache.poi:poi:${versions.poi}" - api "commons-codec:commons-codec:${versions.commonsCodec}" - api "org.apache.xmlbeans:xmlbeans:${versions.xmlbeans}" - api "org.apache.commons:commons-collections4:${versions.commonsCollections4}" + api "org.apache.poi:poi-ooxml:${buildVersions.poi}" + api "org.apache.poi:poi-ooxml-lite:${buildVersions.poi}" + api "org.apache.poi:poi:${buildVersions.poi}" + api "commons-codec:commons-codec:${buildVersions.commonsCodec}" + api "org.apache.xmlbeans:xmlbeans:${buildVersions.xmlbeans}" + api "org.apache.commons:commons-collections4:${buildVersions.commonsCollections4}" // MS Office - api "org.apache.poi:poi-scratchpad:${versions.poi}" + api "org.apache.poi:poi-scratchpad:${buildVersions.poi}" // Apple iWork - api "org.apache.commons:commons-compress:${versions.commonsCompress}" + api "org.apache.commons:commons-compress:${buildVersions.commonsCompress}" // Outlook documents - api "org.apache.james:apache-mime4j-core:${versions.mime4j}" - api "org.apache.james:apache-mime4j-dom:${versions.mime4j}" + api "org.apache.james:apache-mime4j-core:${buildVersions.mime4j}" + api "org.apache.james:apache-mime4j-dom:${buildVersions.mime4j}" // EPUB books - api "org.apache.commons:commons-lang3:${versions.commonsLang3}" + api "org.apache.commons:commons-lang3:${buildVersions.commonsLang3}" // Microsoft Word files with visio diagrams - api "org.apache.commons:commons-math3:${versions.commonsMath3}" + api "org.apache.commons:commons-math3:${buildVersions.commonsMath3}" // POIs dependency - api "com.zaxxer:SparseBitSet:${versions.sparsebitset}" + api "com.zaxxer:SparseBitSet:${buildVersions.sparsebitset}" } restResources { diff --git a/server/build.gradle b/server/build.gradle index 1afb32a973e02..f995c297a7726 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -11,6 +11,8 @@ apply plugin: 'elasticsearch.build' apply plugin: 'elasticsearch.publish' apply plugin: 'elasticsearch.internal-cluster-test' apply plugin: 'elasticsearch.internal-test-artifact' +apply plugin: 'elasticsearch.internal-java-rest-test' + publishing { publications { @@ -36,6 +38,7 @@ dependencies { api project(":libs:plugin-analysis-api") api project(':libs:grok') api project(":libs:tdigest") + implementation project(":libs:arrow") implementation project(":libs:simdvec") implementation project(":libs:entitlement") @@ -81,6 +84,8 @@ dependencies { internalClusterTestImplementation(project(':modules:reindex')) internalClusterTestImplementation(project(':modules:mapper-extras')) internalClusterTestImplementation(project(':modules:data-streams')) + + javaRestTestImplementation(project(":libs:arrow")) } spotless { @@ -286,3 +291,7 @@ tasks.named("licenseHeaders").configure { } tasks.withType(Checkstyle.class).configureEach { t -> t.getMaxHeapSize().set("2g") } + +tasks.named('javaRestTest') { + usesDefaultDistribution() +} diff --git a/server/src/javaRestTest/java/org/elasticsearch/action/bulk/arrow/BulkArrowIT.java b/server/src/javaRestTest/java/org/elasticsearch/action/bulk/arrow/BulkArrowIT.java new file mode 100644 index 0000000000000..8f46bde3945c0 --- /dev/null +++ b/server/src/javaRestTest/java/org/elasticsearch/action/bulk/arrow/BulkArrowIT.java @@ -0,0 +1,182 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.action.bulk.arrow; + +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.ipc.ArrowStreamWriter; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.FieldType; +import org.apache.arrow.vector.types.pojo.Schema; +import org.apache.arrow.vector.util.Text; +import org.apache.http.entity.ByteArrayEntity; +import org.apache.http.entity.ContentType; +import org.elasticsearch.arrow.Arrow; +import org.elasticsearch.client.Request; +import org.elasticsearch.test.cluster.ElasticsearchCluster; +import org.elasticsearch.test.cluster.local.distribution.DistributionType; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xcontent.XContentType; +import org.junit.ClassRule; + +import java.io.ByteArrayOutputStream; +import java.util.List; + +/** + * End-to-end test for Arrow bulk ingestion. Tests for the various Arrow datatypes and + * bulk actions are in {@code ArrowBulkIncrementalParserTests} + */ +public class BulkArrowIT extends ESRestTestCase { + + @ClassRule + public static ElasticsearchCluster cluster = ElasticsearchCluster.local() + .distribution(DistributionType.INTEG_TEST) + .setting("xpack.security.enabled", "false") + .setting("xpack.license.self_generated.type", "basic") + .build(); + + @Override + protected String getTestRestCluster() { + return cluster.getHttpAddresses(); + } + + public void testBulk() throws Exception { + + // Create a dataframe with two columns: integer and string + Field intField = new Field("ints", FieldType.nullable(new ArrowType.Int(32, true)), null); + Field strField = new Field("strings", FieldType.nullable(new ArrowType.Utf8()), null); + Schema schema = new Schema(List.of(intField, strField)); + + int batchCount = 7; + int rowCount = 11; + + byte[] payload; + + // Create vectors and write them to a byte array + try ( + var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); + var root = VectorSchemaRoot.create(schema, allocator); + ) { + var baos = new ByteArrayOutputStream(); + IntVector intVector = (IntVector) root.getVector(0); + VarCharVector stringVector = (VarCharVector) root.getVector(1); + + try (ArrowStreamWriter writer = new ArrowStreamWriter(root, null, baos)) { + for (int batch = 0; batch < batchCount; batch++) { + intVector.allocateNew(rowCount); + stringVector.allocateNew(rowCount); + for (int row = 0; row < rowCount; row++) { + int globalRow = row + batch * rowCount; + intVector.set(row, globalRow); + stringVector.set(row, new Text("row" + globalRow)); + } + root.setRowCount(rowCount); + writer.writeBatch(); + } + } + payload = baos.toByteArray(); + } + + { + // Bulk insert the arrow stream + var request = new Request("POST", "/arrow_bulk_test/_bulk"); + request.addParameter("refresh", "wait_for"); + request.setEntity(new ByteArrayEntity(payload, ContentType.create(Arrow.MEDIA_TYPE))); + + var response = client().performRequest(request); + var result = XContentType.JSON.xContent() + .createParser(XContentParserConfiguration.EMPTY, response.getEntity().getContent()) + .map(); + + assertEquals(Boolean.FALSE, result.get("errors")); + assertEquals(batchCount * rowCount, ((List)result.get("items")).size()); + } + + { + // Check that the index effectively contains what we sent + var request = new Request("GET", "/arrow_bulk_test/_count"); + var response = client().performRequest(request); + var result = XContentType.JSON.xContent() + .createParser(XContentParserConfiguration.EMPTY, response.getEntity().getContent()) + .map(); + + assertEquals(batchCount * rowCount, result.get("count")); + } + } + + public void testDictionary() throws Exception { + + //DictionaryEncoding encoding = new DictionaryEncoding() + + // Create a dataframe with two columns: integer and string + Field intField = new Field("ints", FieldType.nullable(new ArrowType.Int(32, true)), null); + Field strField = new Field("strings", FieldType.nullable(new ArrowType.Utf8()), null); + Schema schema = new Schema(List.of(intField, strField)); + + int batchCount = 7; + int rowCount = 11; + + byte[] payload; + + // Create vectors and write them to a byte array + try ( + var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); + var root = VectorSchemaRoot.create(schema, allocator); + ) { + var baos = new ByteArrayOutputStream(); + IntVector intVector = (IntVector) root.getVector(0); + VarCharVector stringVector = (VarCharVector) root.getVector(1); + + try (ArrowStreamWriter writer = new ArrowStreamWriter(root, null, baos)) { + for (int batch = 0; batch < batchCount; batch++) { + intVector.allocateNew(rowCount); + stringVector.allocateNew(rowCount); + for (int row = 0; row < rowCount; row++) { + int globalRow = row + batch * rowCount; + intVector.set(row, globalRow); + stringVector.set(row, new Text("row" + globalRow)); + } + root.setRowCount(rowCount); + writer.writeBatch(); + } + } + payload = baos.toByteArray(); + } + + { + // Bulk insert the arrow stream + var request = new Request("POST", "/arrow_bulk_test/_bulk"); + request.addParameter("refresh", "wait_for"); + request.setEntity(new ByteArrayEntity(payload, ContentType.create(Arrow.MEDIA_TYPE))); + + var response = client().performRequest(request); + var result = XContentType.JSON.xContent() + .createParser(XContentParserConfiguration.EMPTY, response.getEntity().getContent()) + .map(); + + assertEquals(Boolean.FALSE, result.get("errors")); + assertEquals(batchCount * rowCount, ((List)result.get("items")).size()); + } + + { + // Check that the index effectively contains what we sent + var request = new Request("GET", "/arrow_bulk_test/_count"); + var response = client().performRequest(request); + var result = XContentType.JSON.xContent() + .createParser(XContentParserConfiguration.EMPTY, response.getEntity().getContent()) + .map(); + + assertEquals(batchCount * rowCount, result.get("count")); + } + } +} diff --git a/server/src/main/java/module-info.java b/server/src/main/java/module-info.java index 86de9b9d8bc13..1fac6d811a47d 100644 --- a/server/src/main/java/module-info.java +++ b/server/src/main/java/module-info.java @@ -16,17 +16,13 @@ requires java.security.jgss; requires java.sql; requires java.management; - requires jdk.unsupported; requires java.net.http; // required by ingest-geoip's dependency maxmind.geoip2 https://github.com/elastic/elasticsearch/issues/93553 requires org.elasticsearch.cli; - requires org.elasticsearch.base; requires org.elasticsearch.nativeaccess; requires org.elasticsearch.geo; requires org.elasticsearch.lz4; requires org.elasticsearch.securesm; - requires org.elasticsearch.xcontent; - requires org.elasticsearch.logging; requires org.elasticsearch.plugin; requires org.elasticsearch.plugin.analysis; requires org.elasticsearch.grok; @@ -55,6 +51,10 @@ requires org.apache.lucene.queryparser; requires org.apache.lucene.sandbox; requires org.apache.lucene.suggest; + requires org.elasticsearch.arrow; + requires org.elasticsearch.xcontent; + requires org.elasticsearch.base; + requires org.elasticsearch.logging; exports org.elasticsearch; exports org.elasticsearch.action; diff --git a/server/src/main/java/org/elasticsearch/action/bulk/AbstractBulkRequestParser.java b/server/src/main/java/org/elasticsearch/action/bulk/AbstractBulkRequestParser.java new file mode 100644 index 0000000000000..9e9c72b337c8c --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/bulk/AbstractBulkRequestParser.java @@ -0,0 +1,75 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.action.bulk; + +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.search.fetch.subphase.FetchSourceContext; +import org.elasticsearch.xcontent.XContentType; + +import java.io.IOException; +import java.util.function.BiConsumer; +import java.util.function.Consumer; + +public abstract class AbstractBulkRequestParser { + + /** + * A parser for streamed data. Every call to {@link #parse(BytesReference, boolean)} should + * consume as much data as possible and return the number of bytes that were consumed. + */ + public interface IncrementalParser extends Releasable { + /** + * @param data the data + * @param lastData is there more data that can be read? + * @return the number of bytes that were parsed + */ + int parse(BytesReference data, boolean lastData) throws IOException; + } + + /** + * Parse the provided {@code data} assuming the provided default values. Index requests + * will be passed to the {@code indexRequestConsumer}, update requests to the + * {@code updateRequestConsumer} and delete requests to the {@code deleteRequestConsumer}. + */ + public abstract void parse( + BytesReference data, + @Nullable String defaultIndex, + @Nullable String defaultRouting, + @Nullable FetchSourceContext defaultFetchSourceContext, + @Nullable String defaultPipeline, + @Nullable Boolean defaultRequireAlias, + @Nullable Boolean defaultRequireDataStream, + @Nullable Boolean defaultListExecutedPipelines, + boolean allowExplicitIndex, + XContentType xContentType, + BiConsumer indexRequestConsumer, + Consumer updateRequestConsumer, + Consumer deleteRequestConsumer + ) throws IOException; + + public abstract IncrementalParser incrementalParser( + @Nullable String defaultIndex, + @Nullable String defaultRouting, + @Nullable FetchSourceContext defaultFetchSourceContext, + @Nullable String defaultPipeline, + @Nullable Boolean defaultRequireAlias, + @Nullable Boolean defaultRequireDataStream, + @Nullable Boolean defaultListExecutedPipelines, + boolean allowExplicitIndex, + XContentType xContentType, + BiConsumer indexRequestConsumer, + Consumer updateRequestConsumer, + Consumer deleteRequestConsumer + ); +} diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java index 2f336566953ba..27f76d9a8ec29 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java @@ -43,31 +43,31 @@ /** * Helper to parse bulk requests. This should be considered an internal class. */ -public final class BulkRequestParser { +public final class BulkRequestParser extends AbstractBulkRequestParser { @UpdateForV10(owner = UpdateForV10.Owner.DATA_MANAGEMENT) // Remove deprecation logger when its usages in checkBulkActionIsProperlyClosed are removed private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(BulkRequestParser.class); - private static final Set SUPPORTED_ACTIONS = Set.of("create", "index", "update", "delete"); + public static final Set SUPPORTED_ACTIONS = Set.of("create", "index", "update", "delete"); private static final String STRICT_ACTION_PARSING_WARNING_KEY = "bulk_request_strict_action_parsing"; - private static final ParseField INDEX = new ParseField("_index"); - private static final ParseField TYPE = new ParseField("_type"); - private static final ParseField ID = new ParseField("_id"); - private static final ParseField ROUTING = new ParseField("routing"); - private static final ParseField OP_TYPE = new ParseField("op_type"); - private static final ParseField VERSION = new ParseField("version"); - private static final ParseField VERSION_TYPE = new ParseField("version_type"); - private static final ParseField RETRY_ON_CONFLICT = new ParseField("retry_on_conflict"); - private static final ParseField PIPELINE = new ParseField("pipeline"); - private static final ParseField SOURCE = new ParseField("_source"); - private static final ParseField IF_SEQ_NO = new ParseField("if_seq_no"); - private static final ParseField IF_PRIMARY_TERM = new ParseField("if_primary_term"); - private static final ParseField REQUIRE_ALIAS = new ParseField(DocWriteRequest.REQUIRE_ALIAS); - private static final ParseField REQUIRE_DATA_STREAM = new ParseField(DocWriteRequest.REQUIRE_DATA_STREAM); - private static final ParseField LIST_EXECUTED_PIPELINES = new ParseField(DocWriteRequest.LIST_EXECUTED_PIPELINES); - private static final ParseField DYNAMIC_TEMPLATES = new ParseField("dynamic_templates"); + public static final ParseField INDEX = new ParseField("_index"); + public static final ParseField TYPE = new ParseField("_type"); + public static final ParseField ID = new ParseField("_id"); + public static final ParseField ROUTING = new ParseField("routing"); + public static final ParseField OP_TYPE = new ParseField("op_type"); + public static final ParseField VERSION = new ParseField("version"); + public static final ParseField VERSION_TYPE = new ParseField("version_type"); + public static final ParseField RETRY_ON_CONFLICT = new ParseField("retry_on_conflict"); + public static final ParseField PIPELINE = new ParseField("pipeline"); + public static final ParseField SOURCE = new ParseField("_source"); + public static final ParseField IF_SEQ_NO = new ParseField("if_seq_no"); + public static final ParseField IF_PRIMARY_TERM = new ParseField("if_primary_term"); + public static final ParseField REQUIRE_ALIAS = new ParseField(DocWriteRequest.REQUIRE_ALIAS); + public static final ParseField REQUIRE_DATA_STREAM = new ParseField(DocWriteRequest.REQUIRE_DATA_STREAM); + public static final ParseField LIST_EXECUTED_PIPELINES = new ParseField(DocWriteRequest.LIST_EXECUTED_PIPELINES); + public static final ParseField DYNAMIC_TEMPLATES = new ParseField("dynamic_templates"); // TODO: Remove this parameter once the BulkMonitoring endpoint has been removed // for CompatibleApi V7 this means to deprecate on type, for V8+ it means to throw an error @@ -127,6 +127,7 @@ private static BytesReference sliceTrimmingCarriageReturn( * will be passed to the {@code indexRequestConsumer}, update requests to the * {@code updateRequestConsumer} and delete requests to the {@code deleteRequestConsumer}. */ + @Override public void parse( BytesReference data, @Nullable String defaultIndex, @@ -142,7 +143,7 @@ public void parse( Consumer updateRequestConsumer, Consumer deleteRequestConsumer ) throws IOException { - IncrementalParser incrementalParser = new IncrementalParser( + IncrementalParser incrementalParser = new XContentIncrementalParser( defaultIndex, defaultRouting, defaultFetchSourceContext, @@ -151,7 +152,9 @@ public void parse( defaultRequireDataStream, defaultListExecutedPipelines, allowExplicitIndex, + deprecateOrErrorOnType, xContentType, + config, indexRequestConsumer, updateRequestConsumer, deleteRequestConsumer @@ -160,6 +163,7 @@ public void parse( incrementalParser.parse(data, true); } + @Override public IncrementalParser incrementalParser( @Nullable String defaultIndex, @Nullable String defaultRouting, @@ -174,7 +178,7 @@ public IncrementalParser incrementalParser( Consumer updateRequestConsumer, Consumer deleteRequestConsumer ) { - return new IncrementalParser( + return new XContentIncrementalParser( defaultIndex, defaultRouting, defaultFetchSourceContext, @@ -183,21 +187,23 @@ public IncrementalParser incrementalParser( defaultRequireDataStream, defaultListExecutedPipelines, allowExplicitIndex, + deprecateOrErrorOnType, xContentType, + config, indexRequestConsumer, updateRequestConsumer, deleteRequestConsumer ); } - public class IncrementalParser { + public static class XContentIncrementalParser implements IncrementalParser { // Bulk requests can contain a lot of repeated strings for the index, pipeline and routing parameters. This map is used to // deduplicate duplicate strings parsed for these parameters. While it does not prevent instantiating the duplicate strings, it // reduces their lifetime to the lifetime of this parse call instead of the lifetime of the full bulk request. private final Map stringDeduplicator = new HashMap<>(); - private final String defaultIndex; + protected final String defaultIndex; private final String defaultRouting; private final FetchSourceContext defaultFetchSourceContext; private final String defaultPipeline; @@ -205,12 +211,14 @@ public class IncrementalParser { private final Boolean defaultRequireDataStream; private final Boolean defaultListExecutedPipelines; private final boolean allowExplicitIndex; + private final boolean deprecateOrErrorOnType; private final XContentType xContentType; + private final XContentParserConfiguration config; private final byte marker; - private final BiConsumer indexRequestConsumer; - private final Consumer updateRequestConsumer; - private final Consumer deleteRequestConsumer; + protected final BiConsumer indexRequestConsumer; + protected final Consumer updateRequestConsumer; + protected final Consumer deleteRequestConsumer; private Exception failure = null; private int incrementalFromOffset = 0; @@ -222,7 +230,7 @@ public class IncrementalParser { private boolean currentListExecutedPipelines = false; private FetchSourceContext currentFetchSourceContext = null; - private IncrementalParser( + protected XContentIncrementalParser( @Nullable String defaultIndex, @Nullable String defaultRouting, @Nullable FetchSourceContext defaultFetchSourceContext, @@ -231,7 +239,9 @@ private IncrementalParser( @Nullable Boolean defaultRequireDataStream, @Nullable Boolean defaultListExecutedPipelines, boolean allowExplicitIndex, + boolean deprecateOrErrorOnType, XContentType xContentType, + XContentParserConfiguration config, BiConsumer indexRequestConsumer, Consumer updateRequestConsumer, Consumer deleteRequestConsumer @@ -244,13 +254,16 @@ private IncrementalParser( this.defaultRequireDataStream = defaultRequireDataStream; this.defaultListExecutedPipelines = defaultListExecutedPipelines; this.allowExplicitIndex = allowExplicitIndex; + this.deprecateOrErrorOnType = deprecateOrErrorOnType; this.xContentType = xContentType; - this.marker = xContentType.xContent().bulkSeparator(); + this.config = config; + this.marker = xContentType == null ? 0 : xContentType.xContent().bulkSeparator(); // null for Arrow this.indexRequestConsumer = indexRequestConsumer; this.updateRequestConsumer = updateRequestConsumer; this.deleteRequestConsumer = deleteRequestConsumer; } + @Override public int parse(BytesReference data, boolean lastData) throws IOException { if (failure != null) { assert false : failure.getMessage(); @@ -264,6 +277,11 @@ public int parse(BytesReference data, boolean lastData) throws IOException { } } + @Override + public void close() { + // Nothing + } + private int tryParse(BytesReference data, boolean lastData) throws IOException { int from = 0; int consumed = 0; @@ -299,228 +317,234 @@ private int tryParse(BytesReference data, boolean lastData) throws IOException { private boolean parseActionLine(BytesReference data, int from, int to) throws IOException { assert currentRequest == null; + try (XContentParser parser = createParser(xContentType.xContent(), config, data, from, to)) { + currentRequest = parseActionLine(parser); + return currentRequest != null; + } + } + + protected DocWriteRequest parseActionLine(XContentParser parser) throws IOException { + + DocWriteRequest currentRequest = null; + // Reset the fields which are accessed during document line parsing currentType = null; currentPipeline = defaultPipeline; currentListExecutedPipelines = defaultListExecutedPipelines != null && defaultListExecutedPipelines; currentFetchSourceContext = defaultFetchSourceContext; - try (XContentParser parser = createParser(xContentType.xContent(), data, from, to)) { + // Move to START_OBJECT + XContentParser.Token token = parser.nextToken(); + if (token == null) { + return null; + } + if (token != XContentParser.Token.START_OBJECT) { + throw new IllegalArgumentException( + "Malformed action/metadata line [" + + line + + "], expected " + + XContentParser.Token.START_OBJECT + + " but found [" + + token + + "]" + ); + } + // Move to FIELD_NAME, that's the action + token = parser.nextToken(); + if (token != XContentParser.Token.FIELD_NAME) { + throw new IllegalArgumentException( + "Malformed action/metadata line [" + + line + + "], expected " + + XContentParser.Token.FIELD_NAME + + " but found [" + + token + + "]" + ); + } + String action = parser.currentName(); + if (SUPPORTED_ACTIONS.contains(action) == false) { + throw new IllegalArgumentException( + "Malformed action/metadata line [" + + line + + "], expected field [create], [delete], [index] or [update] but found [" + + action + + "]" + ); + } - // Move to START_OBJECT - XContentParser.Token token = parser.nextToken(); - if (token == null) { - return false; - } - if (token != XContentParser.Token.START_OBJECT) { - throw new IllegalArgumentException( - "Malformed action/metadata line [" - + line - + "], expected " - + XContentParser.Token.START_OBJECT - + " but found [" - + token - + "]" - ); - } - // Move to FIELD_NAME, that's the action - token = parser.nextToken(); - if (token != XContentParser.Token.FIELD_NAME) { - throw new IllegalArgumentException( - "Malformed action/metadata line [" - + line - + "], expected " - + XContentParser.Token.FIELD_NAME - + " but found [" - + token - + "]" - ); - } - String action = parser.currentName(); - if (SUPPORTED_ACTIONS.contains(action) == false) { - throw new IllegalArgumentException( - "Malformed action/metadata line [" - + line - + "], expected field [create], [delete], [index] or [update] but found [" - + action - + "]" - ); - } + String index = defaultIndex; + String id = null; + String routing = defaultRouting; + String opType = null; + long version = Versions.MATCH_ANY; + VersionType versionType = VersionType.INTERNAL; + long ifSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; + long ifPrimaryTerm = UNASSIGNED_PRIMARY_TERM; + int retryOnConflict = 0; + boolean requireAlias = defaultRequireAlias != null && defaultRequireAlias; + boolean requireDataStream = defaultRequireDataStream != null && defaultRequireDataStream; + Map dynamicTemplates = Map.of(); + + // at this stage, next token can either be END_OBJECT (and use default index and type, with auto generated id) + // or START_OBJECT which will have another set of parameters + token = parser.nextToken(); - String index = defaultIndex; - String id = null; - String routing = defaultRouting; - String opType = null; - long version = Versions.MATCH_ANY; - VersionType versionType = VersionType.INTERNAL; - long ifSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; - long ifPrimaryTerm = UNASSIGNED_PRIMARY_TERM; - int retryOnConflict = 0; - boolean requireAlias = defaultRequireAlias != null && defaultRequireAlias; - boolean requireDataStream = defaultRequireDataStream != null && defaultRequireDataStream; - Map dynamicTemplates = Map.of(); - - // at this stage, next token can either be END_OBJECT (and use default index and type, with auto generated id) - // or START_OBJECT which will have another set of parameters - token = parser.nextToken(); - - if (token == XContentParser.Token.START_OBJECT) { - String currentFieldName = null; - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - currentFieldName = parser.currentName(); - } else if (token.isValue()) { - if (INDEX.match(currentFieldName, parser.getDeprecationHandler())) { - if (allowExplicitIndex == false) { - throw new IllegalArgumentException("explicit index in bulk is not allowed"); - } - index = stringDeduplicator.computeIfAbsent(parser.text(), Function.identity()); - } else if (TYPE.match(currentFieldName, parser.getDeprecationHandler())) { - if (deprecateOrErrorOnType) { - throw new IllegalArgumentException( - "Action/metadata line [" + line + "] contains an unknown parameter [" + currentFieldName + "]" - ); - } - currentType = stringDeduplicator.computeIfAbsent(parser.text(), Function.identity()); - } else if (ID.match(currentFieldName, parser.getDeprecationHandler())) { - id = parser.text(); - } else if (ROUTING.match(currentFieldName, parser.getDeprecationHandler())) { - routing = stringDeduplicator.computeIfAbsent(parser.text(), Function.identity()); - } else if (OP_TYPE.match(currentFieldName, parser.getDeprecationHandler())) { - opType = parser.text(); - } else if (VERSION.match(currentFieldName, parser.getDeprecationHandler())) { - version = parser.longValue(); - } else if (VERSION_TYPE.match(currentFieldName, parser.getDeprecationHandler())) { - versionType = VersionType.fromString(parser.text()); - } else if (IF_SEQ_NO.match(currentFieldName, parser.getDeprecationHandler())) { - ifSeqNo = parser.longValue(); - } else if (IF_PRIMARY_TERM.match(currentFieldName, parser.getDeprecationHandler())) { - ifPrimaryTerm = parser.longValue(); - } else if (RETRY_ON_CONFLICT.match(currentFieldName, parser.getDeprecationHandler())) { - retryOnConflict = parser.intValue(); - } else if (PIPELINE.match(currentFieldName, parser.getDeprecationHandler())) { - currentPipeline = stringDeduplicator.computeIfAbsent(parser.text(), Function.identity()); - } else if (SOURCE.match(currentFieldName, parser.getDeprecationHandler())) { - currentFetchSourceContext = FetchSourceContext.fromXContent(parser); - } else if (REQUIRE_ALIAS.match(currentFieldName, parser.getDeprecationHandler())) { - requireAlias = parser.booleanValue(); - } else if (REQUIRE_DATA_STREAM.match(currentFieldName, parser.getDeprecationHandler())) { - requireDataStream = parser.booleanValue(); - } else if (LIST_EXECUTED_PIPELINES.match(currentFieldName, parser.getDeprecationHandler())) { - currentListExecutedPipelines = parser.booleanValue(); - } else { + if (token == XContentParser.Token.START_OBJECT) { + String currentFieldName = null; + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token.isValue()) { + if (INDEX.match(currentFieldName, parser.getDeprecationHandler())) { + if (allowExplicitIndex == false) { + throw new IllegalArgumentException("explicit index in bulk is not allowed"); + } + index = stringDeduplicator.computeIfAbsent(parser.text(), Function.identity()); + } else if (TYPE.match(currentFieldName, parser.getDeprecationHandler())) { + if (deprecateOrErrorOnType) { throw new IllegalArgumentException( "Action/metadata line [" + line + "] contains an unknown parameter [" + currentFieldName + "]" ); } - } else if (token == XContentParser.Token.START_ARRAY) { + currentType = stringDeduplicator.computeIfAbsent(parser.text(), Function.identity()); + } else if (ID.match(currentFieldName, parser.getDeprecationHandler())) { + id = parser.text(); + } else if (ROUTING.match(currentFieldName, parser.getDeprecationHandler())) { + routing = stringDeduplicator.computeIfAbsent(parser.text(), Function.identity()); + } else if (OP_TYPE.match(currentFieldName, parser.getDeprecationHandler())) { + opType = parser.text(); + } else if (VERSION.match(currentFieldName, parser.getDeprecationHandler())) { + version = parser.longValue(); + } else if (VERSION_TYPE.match(currentFieldName, parser.getDeprecationHandler())) { + versionType = VersionType.fromString(parser.text()); + } else if (IF_SEQ_NO.match(currentFieldName, parser.getDeprecationHandler())) { + ifSeqNo = parser.longValue(); + } else if (IF_PRIMARY_TERM.match(currentFieldName, parser.getDeprecationHandler())) { + ifPrimaryTerm = parser.longValue(); + } else if (RETRY_ON_CONFLICT.match(currentFieldName, parser.getDeprecationHandler())) { + retryOnConflict = parser.intValue(); + } else if (PIPELINE.match(currentFieldName, parser.getDeprecationHandler())) { + currentPipeline = stringDeduplicator.computeIfAbsent(parser.text(), Function.identity()); + } else if (SOURCE.match(currentFieldName, parser.getDeprecationHandler())) { + currentFetchSourceContext = FetchSourceContext.fromXContent(parser); + } else if (REQUIRE_ALIAS.match(currentFieldName, parser.getDeprecationHandler())) { + requireAlias = parser.booleanValue(); + } else if (REQUIRE_DATA_STREAM.match(currentFieldName, parser.getDeprecationHandler())) { + requireDataStream = parser.booleanValue(); + } else if (LIST_EXECUTED_PIPELINES.match(currentFieldName, parser.getDeprecationHandler())) { + currentListExecutedPipelines = parser.booleanValue(); + } else { throw new IllegalArgumentException( - "Malformed action/metadata line [" - + line - + "], expected a simple value for field [" - + currentFieldName - + "] but found [" - + token - + "]" + "Action/metadata line [" + line + "] contains an unknown parameter [" + currentFieldName + "]" ); + } + } else if (token == XContentParser.Token.START_ARRAY) { + throw new IllegalArgumentException( + "Malformed action/metadata line [" + + line + + "], expected a simple value for field [" + + currentFieldName + + "] but found [" + + token + + "]" + ); + } else if (token == XContentParser.Token.START_OBJECT + && DYNAMIC_TEMPLATES.match(currentFieldName, parser.getDeprecationHandler())) { + dynamicTemplates = parser.mapStrings(); } else if (token == XContentParser.Token.START_OBJECT - && DYNAMIC_TEMPLATES.match(currentFieldName, parser.getDeprecationHandler())) { - dynamicTemplates = parser.mapStrings(); - } else if (token == XContentParser.Token.START_OBJECT - && SOURCE.match(currentFieldName, parser.getDeprecationHandler())) { - currentFetchSourceContext = FetchSourceContext.fromXContent(parser); - } else if (token != XContentParser.Token.VALUE_NULL) { - throw new IllegalArgumentException( - "Malformed action/metadata line [" - + line - + "], expected a simple value for field [" - + currentFieldName - + "] but found [" - + token - + "]" - ); - } - } - } else if (token != XContentParser.Token.END_OBJECT) { + && SOURCE.match(currentFieldName, parser.getDeprecationHandler())) { + currentFetchSourceContext = FetchSourceContext.fromXContent(parser); + } else if (token != XContentParser.Token.VALUE_NULL) { + throw new IllegalArgumentException( + "Malformed action/metadata line [" + + line + + "], expected a simple value for field [" + + currentFieldName + + "] but found [" + + token + + "]" + ); + } + } + } else if (token != XContentParser.Token.END_OBJECT) { + throw new IllegalArgumentException( + "Malformed action/metadata line [" + + line + + "], expected " + + XContentParser.Token.START_OBJECT + + " or " + + XContentParser.Token.END_OBJECT + + " but found [" + + token + + "]" + ); + } + checkBulkActionIsProperlyClosed(parser, config, line); + + if ("delete".equals(action)) { + if (dynamicTemplates.isEmpty() == false) { throw new IllegalArgumentException( - "Malformed action/metadata line [" - + line - + "], expected " - + XContentParser.Token.START_OBJECT - + " or " - + XContentParser.Token.END_OBJECT - + " but found [" - + token - + "]" + "Delete request in line [" + line + "] does not accept " + DYNAMIC_TEMPLATES.getPreferredName() ); } - checkBulkActionIsProperlyClosed(parser, line); - - if ("delete".equals(action)) { + currentRequest = new DeleteRequest(index).id(id) + .routing(routing) + .version(version) + .versionType(versionType) + .setIfSeqNo(ifSeqNo) + .setIfPrimaryTerm(ifPrimaryTerm); + } else { + // we use internalAdd so we don't fork here, this allows us not to copy over the big byte array to small chunks + // of index request. + if ("index".equals(action) || "create".equals(action)) { + var indexRequest = new IndexRequest(index).id(id) + .routing(routing) + .version(version) + .versionType(versionType) + .setPipeline(currentPipeline) + .setIfSeqNo(ifSeqNo) + .setIfPrimaryTerm(ifPrimaryTerm) + .setDynamicTemplates(dynamicTemplates) + .setRequireAlias(requireAlias) + .setRequireDataStream(requireDataStream) + .setListExecutedPipelines(currentListExecutedPipelines) + .setIncludeSourceOnError(config.includeSourceOnError());if ("create".equals(action)) { + indexRequest = indexRequest.create(true); + } else if (opType != null) { + indexRequest = indexRequest.create("create".equals(opType)); + } + currentRequest = indexRequest; + } else if ("update".equals(action)) { + if (version != Versions.MATCH_ANY || versionType != VersionType.INTERNAL) { + throw new IllegalArgumentException( + "Update requests do not support versioning. " + "Please use `if_seq_no` and `if_primary_term` instead" + ); + } + if (requireDataStream) { + throw new IllegalArgumentException( + "Update requests do not support the `require_data_stream` flag, " + + "as data streams do not support update operations" + ); + } + // TODO: support dynamic_templates in update requests if (dynamicTemplates.isEmpty() == false) { throw new IllegalArgumentException( - "Delete request in line [" + line + "] does not accept " + DYNAMIC_TEMPLATES.getPreferredName() + "Update request in line [" + line + "] does not accept " + DYNAMIC_TEMPLATES.getPreferredName() ); } - currentRequest = new DeleteRequest(index).id(id) + UpdateRequest updateRequest = new UpdateRequest().index(index) + .id(id) .routing(routing) - .version(version) - .versionType(versionType) + .retryOnConflict(retryOnConflict) .setIfSeqNo(ifSeqNo) - .setIfPrimaryTerm(ifPrimaryTerm); - } else { - // we use internalAdd so we don't fork here, this allows us not to copy over the big byte array to small chunks - // of index request. - if ("index".equals(action) || "create".equals(action)) { - var indexRequest = new IndexRequest(index).id(id) - .routing(routing) - .version(version) - .versionType(versionType) - .setPipeline(currentPipeline) - .setIfSeqNo(ifSeqNo) - .setIfPrimaryTerm(ifPrimaryTerm) - .setDynamicTemplates(dynamicTemplates) - .setRequireAlias(requireAlias) - .setRequireDataStream(requireDataStream) - .setListExecutedPipelines(currentListExecutedPipelines) - .setIncludeSourceOnError(config.includeSourceOnError()); - if ("create".equals(action)) { - indexRequest = indexRequest.create(true); - } else if (opType != null) { - indexRequest = indexRequest.create("create".equals(opType)); - } - currentRequest = indexRequest; - } else if ("update".equals(action)) { - if (version != Versions.MATCH_ANY || versionType != VersionType.INTERNAL) { - throw new IllegalArgumentException( - "Update requests do not support versioning. " + "Please use `if_seq_no` and `if_primary_term` instead" - ); - } - if (requireDataStream) { - throw new IllegalArgumentException( - "Update requests do not support the `require_data_stream` flag, " - + "as data streams do not support update operations" - ); - } - // TODO: support dynamic_templates in update requests - if (dynamicTemplates.isEmpty() == false) { - throw new IllegalArgumentException( - "Update request in line [" + line + "] does not accept " + DYNAMIC_TEMPLATES.getPreferredName() - ); - } - UpdateRequest updateRequest = new UpdateRequest().index(index) - .id(id) - .routing(routing) - .retryOnConflict(retryOnConflict) - .setIfSeqNo(ifSeqNo) - .setIfPrimaryTerm(ifPrimaryTerm) - .setRequireAlias(requireAlias) - .routing(routing); - currentRequest = updateRequest; - } + .setIfPrimaryTerm(ifPrimaryTerm) + .setRequireAlias(requireAlias) + .routing(routing); + currentRequest = updateRequest; } } - return true; + return currentRequest; } private void parseAndConsumeDocumentLine(BytesReference data, int from, int to) throws IOException { @@ -532,6 +556,7 @@ private void parseAndConsumeDocumentLine(BytesReference data, int from, int to) try ( XContentParser sliceParser = createParser( xContentType.xContent(), + config, sliceTrimmingCarriageReturn(data, from, to, xContentType) ) ) { @@ -551,7 +576,9 @@ private void parseAndConsumeDocumentLine(BytesReference data, int from, int to) } @UpdateForV10(owner = UpdateForV10.Owner.DATA_MANAGEMENT) // Remove lenient parsing in V8 BWC mode - private void checkBulkActionIsProperlyClosed(XContentParser parser, int line) throws IOException { + private static void checkBulkActionIsProperlyClosed( + XContentParser parser, XContentParserConfiguration config, int line + ) throws IOException { XContentParser.Token token; try { token = parser.nextToken(); @@ -602,9 +629,11 @@ private void checkBulkActionIsProperlyClosed(XContentParser parser, int line) th } } - private XContentParser createParser(XContent xContent, BytesReference data) throws IOException { + private static XContentParser createParser( + XContent xContent, XContentParserConfiguration config, BytesReference data + ) throws IOException { if (data.hasArray()) { - return parseBytesArray(xContent, data, 0, data.length()); + return parseBytesArray(xContent, config, data, 0, data.length()); } else { return xContent.createParser(config, data.streamInput()); } @@ -612,21 +641,25 @@ private XContentParser createParser(XContent xContent, BytesReference data) thro // Create an efficient parser of the given bytes, trying to directly parse a byte array if possible and falling back to stream wrapping // otherwise. - private XContentParser createParser(XContent xContent, BytesReference data, int from, int nextMarker) throws IOException { + private static XContentParser createParser( + XContent xContent, XContentParserConfiguration config, BytesReference data, int from, int nextMarker + ) throws IOException { if (data.hasArray()) { - return parseBytesArray(xContent, data, from, nextMarker); + return parseBytesArray(xContent, config, data, from, nextMarker); } else { final int length = nextMarker - from; final BytesReference slice = data.slice(from, length); if (slice.hasArray()) { - return parseBytesArray(xContent, slice, 0, length); + return parseBytesArray(xContent, config, slice, 0, length); } else { return xContent.createParser(config, slice.streamInput()); } } } - private XContentParser parseBytesArray(XContent xContent, BytesReference array, int from, int nextMarker) throws IOException { + private static XContentParser parseBytesArray( + XContent xContent, XContentParserConfiguration config, BytesReference array, int from, int nextMarker + ) throws IOException { assert array.hasArray(); final int offset = array.arrayOffset(); return xContent.createParser(config, array.array(), offset + from, nextMarker - from); diff --git a/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParser.java b/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParser.java new file mode 100644 index 0000000000000..dbb6101319af0 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParser.java @@ -0,0 +1,357 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.action.bulk.arrow; + +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.BaseIntVector; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VariableWidthFieldVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.complex.MapVector; +import org.apache.arrow.vector.complex.StructVector; +import org.apache.arrow.vector.complex.UnionVector; +import org.apache.arrow.vector.dictionary.Dictionary; +import org.apache.arrow.vector.types.Types; +import org.apache.arrow.vector.util.Text; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BulkRequestParser; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.arrow.Arrow; +import org.elasticsearch.arrow.ArrowFormatException; +import org.elasticsearch.arrow.ArrowToXContent; +import org.elasticsearch.arrow.XContentBuffer; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.search.fetch.subphase.FetchSourceContext; +import org.elasticsearch.xcontent.XContent; +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xcontent.XContentType; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.BitSet; +import java.util.Map; +import java.util.function.BiConsumer; +import java.util.function.Consumer; + + +class ArrowBulkIncrementalParser extends BulkRequestParser.XContentIncrementalParser { + + /** XContent format used to encode source documents */ + private static final XContent SOURCE_XCONTENT = XContentType.CBOR.xContent(); + + private static final String ID = "_id"; + private static final String INDEX = "_index"; + private static final String ACTION = "_bulk_action"; + + private DocWriteRequest.OpType defaultOpType; + + private ArrowIncrementalParser arrowParser; + private BufferAllocator allocator; + private VectorSchemaRoot schemaRoot; + private Map dictionaries; + + private Integer idField = null; + private Integer indexField = null; + private Integer actionField = null; + private BitSet valueFields; + + ArrowBulkIncrementalParser( + DocWriteRequest.OpType defaultOpType, + @Nullable String defaultIndex, + @Nullable String defaultRouting, + @Nullable FetchSourceContext defaultFetchSourceContext, + @Nullable String defaultPipeline, + @Nullable Boolean defaultRequireAlias, + @Nullable Boolean defaultRequireDataStream, + @Nullable Boolean defaultListExecutedPipelines, + boolean allowExplicitIndex, + XContentType xContentType, + XContentParserConfiguration config, + BiConsumer indexRequestConsumer, + Consumer updateRequestConsumer, + Consumer deleteRequestConsumer + ) { + super( + defaultIndex, + defaultRouting, + defaultFetchSourceContext, + defaultPipeline, + defaultRequireAlias, + defaultRequireDataStream, + defaultListExecutedPipelines, + allowExplicitIndex, + true, // deprecateOrErrorOnType + xContentType, + config, + indexRequestConsumer, + updateRequestConsumer, + deleteRequestConsumer + ); + + this.defaultOpType = defaultOpType; + + // FIXME: hard-coded limit to 100 MiB per record batch. Should we add an AllocationListener that calls ES memory management? + this.allocator = Arrow.rootAllocator().newChildAllocator("bulk-ingestion", 0, 100 * 1024 * 1024); + + this.arrowParser = new ArrowIncrementalParser( + new RootAllocator(), + new ArrowIncrementalParser.Listener() { + @Override + public void startStream(VectorSchemaRoot schemaRoot) throws IOException { + startArrowStream(schemaRoot); + } + + @Override + public void nextBatch(Map dictionary) throws IOException { + nextArrowBatch(dictionary); + } + + @Override + public void endStream() throws IOException { + endArrowStream(); + } + } + ); + } + + @Override + public int parse(BytesReference data, boolean lastData) throws IOException { + return arrowParser.parse(data, lastData); + } + + @Override + public void close() { + super.close(); + if (schemaRoot != null) { + schemaRoot.close(); + schemaRoot = null; + } + } + + private void startArrowStream(VectorSchemaRoot root) { + + this.schemaRoot = root; + + var schemaFields = root.getFieldVectors(); + var valueFields = new BitSet(schemaFields.size()); + + for (int i = 0; i < schemaFields.size(); i++) { + var field = schemaFields.get(i); + + switch (field.getName()) { + case ID -> idField = i; + case INDEX -> indexField = i; + case ACTION -> { + var type = field.getMinorType(); + if (type != Types.MinorType.MAP && type != Types.MinorType.STRUCT) { + throw new ArrowFormatException("Field '" + ACTION + "' should be a map or a struct"); + } + actionField = i; + } + // Regular field that will be added to the document. + default -> valueFields.set(i); + } + } + + this.valueFields = valueFields; + } + + private void nextArrowBatch(Map dictionary) throws IOException { + this.dictionaries = dictionary; + int rowCount = schemaRoot.getRowCount(); + FieldVector idVector = idField == null ? null : schemaRoot.getVector(idField); + FieldVector indexVector = indexField == null ? null : schemaRoot.getVector(indexField); + FieldVector actionVector = actionField == null ? null : schemaRoot.getVector(actionField); + + for (int i = 0; i < rowCount; i++) { + String id = idVector == null ? null : getString(idVector, i); + String index = indexVector == null ? null : getString(indexVector, i); + + var action = parseAction(actionVector, i, id, index); + switch (action) { + case IndexRequest ir -> { + ir.source(generateSource(i), SOURCE_XCONTENT.type()); + indexRequestConsumer.accept(ir, null); + } + case UpdateRequest ur -> { + // Script updates aren't supported in Arrow format + ur.doc(generateSource(i), SOURCE_XCONTENT.type()); + updateRequestConsumer.accept(ur); + } + case DeleteRequest dr -> { + deleteRequestConsumer.accept(dr); + } + default -> {} + } + } + } + + protected BytesReference generateSource(int position) throws IOException { + var output = new BytesReferenceOutputStream(); + try(var generator = SOURCE_XCONTENT.createGenerator(output)) { + generator.writeStartObject(); + int rowCount = schemaRoot.getRowCount(); + for (int i = 0; i < rowCount; i++) { + if (valueFields.get(i)) { + ArrowToXContent.writeField(schemaRoot.getVector(i), position, dictionaries, generator); + } + } + generator.writeEndObject(); + } + + return output.asBytesReference(); + } + + private void endArrowStream() { + close(); + } + + // Visible for testing + DocWriteRequest parseAction(@Nullable FieldVector actionVector, int position, String id, String index) throws IOException { + + DocWriteRequest request; + + try (var generator = new XContentBuffer()) { + + if (actionVector == null) { + // Create a `{ defaultOpType: {} }` action + generator.writeStartObject(); + generator.writeFieldName(defaultOpType.getLowercase()); + generator.writeStartObject(); + generator.writeEndObject(); + generator.writeEndObject(); + } else { + String opType = getNamedString(actionVector, "op_type", position); + if (opType == null) { + opType = defaultOpType.getLowercase(); + } + // Create a `{ opType: { properties } }` action + // Note: the "op_type" property may also exist, but the action parser accepts it. + generator.writeStartObject(); + generator.writeFieldName(opType); + ArrowToXContent.writeValue(actionVector, position, dictionaries, generator); + generator.writeEndObject(); + } + + request = parseActionLine(generator.asParser()); + } + + if (id != null) { + if (request.id() != null) { + throw new ArrowFormatException( + "'" + ID + "' found both as top-level field and in '" + ACTION + "' at position [" + position + "]" + ); + } + + switch (request) { + case IndexRequest ir -> ir.id(id); + case UpdateRequest ur -> ur.id(id); + case DeleteRequest ur -> ur.id(id); + default -> throw new IllegalArgumentException("Unknown request type [" + request.opType() + "]"); + } + } + + if (index != null) { + // Testing references on purpose to detect default index passed down to the request + if (request.index() != defaultIndex) { + throw new ArrowFormatException( + "'" + INDEX + "' found both as top-level field and in '" + ACTION + "' at position [" + position + "]" + ); + } + request.index(index); + } + + return request; + } + + private String getNamedString(FieldVector vector, String name, int position) { + byte[] nameBytes = name.getBytes(StandardCharsets.UTF_8); + + if (vector instanceof MapVector mapVector) { + // A Map is a variable-size list of structs with two fields, key and value (in this order) + var data = mapVector.getDataVector(); + var keyVec = (VarCharVector)data.getChildrenFromFields().get(0); + var valueVec = data.getChildrenFromFields().get(1); + + var key = new Text(); + for (int pos = mapVector.getElementStartIndex(position); pos < mapVector.getElementEndIndex(position); pos++) { + keyVec.read(pos, key); + if (Arrays.equals(nameBytes, 0, nameBytes.length, key.getBytes(), 0, (int)key.getLength())) { + return getString(valueVec, pos); + } + } + // Not found + return null; + } + + if (vector instanceof StructVector structVector) { + var childVector = structVector.getChild(name); + return childVector == null ? null : getString(childVector, position); + } + + for (var child: vector.getChildrenFromFields()) { + if (child instanceof ValueVector valueVector && valueVector.getName().equals(name)) { + return getString(valueVector, position); + } + } + return null; + } + + private String getString(ValueVector vector, int position) { + if (vector.isNull(position)) { + return null; + } + + return switch (vector.getMinorType()) { + case TINYINT, SMALLINT, INT, BIGINT, UINT1, UINT2, UINT4, UINT8 -> + String.valueOf(((BaseIntVector)vector).getValueAsLong(position)); + + case VARCHAR, LARGEVARCHAR, VIEWVARCHAR -> { + var bytesVector = (VariableWidthFieldVector)vector; + yield new String(bytesVector.get(position), StandardCharsets.UTF_8); + } + + case UNION -> { + UnionVector unionVector = (UnionVector)vector; + // Find the child field that isn't null, which is the active variant. + for (var variantVec: unionVector.getChildrenFromFields()) { + if (variantVec.isNull(position) == false) { + yield getString(variantVec, position); + } + } + yield null; + } + + default -> { + throw new ArrowFormatException( + "Arrow type [" + vector.getMinorType() + "] cannot be converted to string" + ); + } + }; + } + + /** + * A byte array stream that can be converted to {@code BytesReference} with zero copy. + */ + private static class BytesReferenceOutputStream extends ByteArrayOutputStream { + BytesArray asBytesReference() { + return new BytesArray(buf, 0, count); + } + } +} diff --git a/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkRequestParser.java b/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkRequestParser.java new file mode 100644 index 0000000000000..5f39bda2aaf64 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkRequestParser.java @@ -0,0 +1,124 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.action.bulk.arrow; + +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.AbstractBulkRequestParser; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.arrow.Arrow; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.RestApiVersion; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.search.fetch.subphase.FetchSourceContext; +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xcontent.XContentType; + +import java.io.IOException; +import java.util.function.BiConsumer; +import java.util.function.Consumer; + +public class ArrowBulkRequestParser extends AbstractBulkRequestParser { + + public static boolean isArrowRequest(RestRequest request) { + if (request.getXContentType() != null) { + return false; // We don't have a XContentType for Arrow (yet) + } + var contentType = request.getHeaders().get("Content-Type"); + if (contentType == null || contentType.isEmpty()) { + return false; + } + return contentType.contains(Arrow.MEDIA_TYPE); + } + + private final RestApiVersion apiVersion; + private final DocWriteRequest.OpType defaultOpType; + + public ArrowBulkRequestParser(RestRequest request) { + // Default operation read from the "op_type" query parameter + // We default to create requests as it's safe and versatile: + // - accepts requests with and without an id, + // - if an id is present, ensures we don't accidentally overwrite an existing document, + // - datastreams only accept create operations. + String str = request.param("op_type", DocWriteRequest.OpType.CREATE.getLowercase()); + this.defaultOpType = DocWriteRequest.OpType.fromString(str); + this.apiVersion = request.getRestApiVersion(); + } + + @Override + public void parse( + BytesReference data, + @Nullable String defaultIndex, + @Nullable String defaultRouting, + @Nullable FetchSourceContext defaultFetchSourceContext, + @Nullable String defaultPipeline, + @Nullable Boolean defaultRequireAlias, + @Nullable Boolean defaultRequireDataStream, + @Nullable Boolean defaultListExecutedPipelines, + boolean allowExplicitIndex, + XContentType xContentType, + BiConsumer indexRequestConsumer, + Consumer updateRequestConsumer, + Consumer deleteRequestConsumer + ) throws IOException { + try(IncrementalParser parser = incrementalParser( + defaultIndex, + defaultRouting, + defaultFetchSourceContext, + defaultPipeline, + defaultRequireAlias, + defaultRequireDataStream, + defaultListExecutedPipelines, + allowExplicitIndex, + xContentType, + indexRequestConsumer, + updateRequestConsumer, + deleteRequestConsumer + )) { + parser.parse(data, true); + } + } + + @Override + public IncrementalParser incrementalParser( + @Nullable String defaultIndex, + @Nullable String defaultRouting, + @Nullable FetchSourceContext defaultFetchSourceContext, + @Nullable String defaultPipeline, + @Nullable Boolean defaultRequireAlias, + @Nullable Boolean defaultRequireDataStream, + @Nullable Boolean defaultListExecutedPipelines, + boolean allowExplicitIndex, + XContentType xContentType, + BiConsumer indexRequestConsumer, + Consumer updateRequestConsumer, + Consumer deleteRequestConsumer + ) { + return new ArrowBulkIncrementalParser( + defaultOpType, + defaultIndex, + defaultRouting, + defaultFetchSourceContext, + defaultPipeline, + defaultRequireAlias, + defaultRequireDataStream, + defaultListExecutedPipelines, + allowExplicitIndex, + xContentType, + XContentParserConfiguration.EMPTY.withRestApiVersion(apiVersion), + indexRequestConsumer, + updateRequestConsumer, + deleteRequestConsumer + ); + } + +} diff --git a/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowIncrementalParser.java b/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowIncrementalParser.java new file mode 100644 index 0000000000000..d3d6b8a7ec48d --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowIncrementalParser.java @@ -0,0 +1,159 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.action.bulk.arrow; + +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.dictionary.Dictionary; +import org.apache.arrow.vector.ipc.ArrowStreamReader; +import org.apache.arrow.vector.ipc.ReadChannel; +import org.apache.arrow.vector.ipc.message.MessageMetadataResult; +import org.apache.arrow.vector.ipc.message.MessageSerializer; +import org.elasticsearch.common.bytes.BytesReference; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Map; + +/** + * An incremental reader for Arrow dataframes. + */ +public class ArrowIncrementalParser implements Closeable { + + public interface Listener { + /** + * Start of the Arrow stream. It's the responsibility of the listener to close this vector root, + * as it may need to live longer than the parser. + */ + void startStream(VectorSchemaRoot schemaRoot) throws IOException; + + /** + * A new {@code RecordBatch} was read. Its vectors are available in the {@code VectorSchemaRoot} that + * was passed to {@link #startStream(VectorSchemaRoot)}. + */ + void nextBatch(Map dictionary) throws IOException; + + /** + * Reached the end of the Arrow stream. + */ + void endStream() throws IOException; + } + + private final Listener listener; + private BytesReferenceChannel channel; + private long expectedDataLength; + private ArrowStreamReader reader; + + private static final int PREFIX_LEN = 8; + + public ArrowIncrementalParser(BufferAllocator allocator, Listener listener) { + this.listener = listener; + this.expectedDataLength = PREFIX_LEN; + this.channel = new BytesReferenceChannel(); + this.reader = new ArrowStreamReader(channel, allocator); + } + + /** + * When {@link #parse(BytesReference, boolean)} returns zero, provides the number of bytes + * that are needed to continue parsing the Arrow stream. Note that {@code parse()} can + * return zero multiple times with an increasing expected data length. + */ + public long expectedDataLength() { + return this.expectedDataLength; + } + + @Override + public void close() throws IOException { + if (this.reader != null) { + this.reader.close(); // Will also close channel. + this.channel = null; + this.reader = null; + } + } + + public int parse(BytesReference data, boolean lastData) throws IOException { + int total = 0; + int consumed; + while ((consumed = doParse(data, lastData)) > 0) { + total += consumed; + data = data.slice(consumed, data.length() - consumed); + // Start a new message + expectedDataLength = PREFIX_LEN; + } + return total; + } + + /** + * Parse an Arrow message (metadata + body). If there aren't enough bytes available, return zero. + */ + private int doParse(BytesReference data, boolean lastData) throws IOException { + + if (data.length() < expectedDataLength) { + return 0; + } + + // See https://arrow.apache.org/docs/format/Columnar.html#serialization-and-interprocess-communication-ipc + + var continuation = data.getIntLE(0); + if (continuation != 0xFFFFFFFF) { + throw new IOException("Bad Arrow continuation prefix [" + Integer.toHexString(continuation) + "] prefix"); + } + + var metadataSize = data.getIntLE(4); + + if (metadataSize == 0) { + // End of stream + return PREFIX_LEN; + } + + // FIXME: enforce a hard limit on metadata size? + int trailing = metadataSize % 8; + if (trailing % 8 != 0) { + // padded to 8 bytes + metadataSize += (8 - trailing); + } + + expectedDataLength = PREFIX_LEN + metadataSize; + if (data.length() < expectedDataLength) { + return 0; + } + + // We may expect some data after the metadata, read metadata to find body length. + // The Arrow library doesn't make it easy to read metadata and then the body, so we read + // the metadata once to get the body length (overhead is low since flatbuffers is zero-copy) + ReadChannel ch = new ReadChannel(new BytesReferenceChannel(data)); + MessageMetadataResult metadata = MessageSerializer.readMessage(ch); + // FIXME: enforce a hard limit on body length? + expectedDataLength += metadata.getMessageBodyLength(); + if (data.length() < expectedDataLength) { + return 0; + } + + // We now have enough data to read a batch (message + data) + channel.setData(data, lastData); + long initialBytesRead = reader.bytesRead(); + + if (reader.bytesRead() == 0) { + VectorSchemaRoot root = reader.getVectorSchemaRoot(); + listener.startStream(root); + + } else { + if (reader.loadNextBatch()) { + listener.nextBatch(reader.getDictionaryVectors()); + } else { + expectedDataLength = 0; + listener.endStream(); + close(); + } + } + + return (int)(reader.bytesRead() - initialBytesRead); + } +} diff --git a/server/src/main/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannel.java b/server/src/main/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannel.java new file mode 100644 index 0000000000000..49eaacb3eff75 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannel.java @@ -0,0 +1,85 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.action.bulk.arrow; + +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefIterator; +import org.elasticsearch.common.bytes.BytesReference; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; + +/** + * A {@code ReadableByteChannel} that reads from {@code ByteReference} data. That data + * can be updated, allowing incremental parsing from a single channel. + */ +class BytesReferenceChannel implements ReadableByteChannel { + + private BytesRefIterator iterator; + private BytesRef current; + private int currentOffset; + private int endOffset; + private boolean lastData = false; + + BytesReferenceChannel() { + // Keep zero/null values + } + + BytesReferenceChannel(BytesReference data) throws IOException { + setData(data, true); + } + + void setData(BytesReference data, boolean lastData) throws IOException { + this.lastData = lastData; + this.iterator = data.iterator(); + nextBytesRef(); + } + + private void nextBytesRef() throws IOException { + this.current = iterator.next(); + if (this.current == null) { + this.currentOffset = 0; + this.endOffset = 0; + } else { + this.currentOffset = this.current.offset; + this.endOffset = this.currentOffset + this.current.length; + } + } + + @Override + public int read(ByteBuffer dst) throws IOException { + int written = 0; + int remaining; + while ((remaining = dst.remaining()) > 0 && this.current != null) { + int len = Math.min(remaining, this.endOffset - this.currentOffset); + dst.put(this.current.bytes, this.currentOffset, len); + this.currentOffset += len; + written += len; + + if (this.currentOffset == this.endOffset) { + nextBytesRef(); + } + } + + return written == 0 && lastData ? -1 : written; + } + + @Override + public boolean isOpen() { + return iterator != null; + } + + @Override + public void close() { + iterator = null; + current = null; + } +} diff --git a/server/src/main/java/org/elasticsearch/rest/RestController.java b/server/src/main/java/org/elasticsearch/rest/RestController.java index a7c59e92bfbf2..647fc2ade4596 100644 --- a/server/src/main/java/org/elasticsearch/rest/RestController.java +++ b/server/src/main/java/org/elasticsearch/rest/RestController.java @@ -403,6 +403,7 @@ private void dispatchRequest( final XContentType xContentType = request.getXContentType(); // TODO consider refactoring to handler.supportsContentStream(xContentType). It is only used with JSON and SMILE if (handler.supportsBulkContent() + && xContentType != null // null for Arrow format && XContentType.JSON != xContentType.canonical() && XContentType.SMILE != xContentType.canonical()) { channel.sendResponse( diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java index 068ea4e48db01..f857479bec053 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java @@ -12,10 +12,12 @@ import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.AbstractBulkRequestParser; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkRequestParser; import org.elasticsearch.action.bulk.BulkShardRequest; import org.elasticsearch.action.bulk.IncrementalBulkService; +import org.elasticsearch.action.bulk.arrow.ArrowBulkRequestParser; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.cluster.metadata.DataStream; @@ -82,6 +84,11 @@ public List routes() { ); } + @Override + public boolean mediaTypesValid(RestRequest request) { + return super.mediaTypesValid(request) || ArrowBulkRequestParser.isArrowRequest(request); + } + @Override public String getName() { return "bulk_action"; @@ -161,8 +168,14 @@ static class ChunkHandler implements BaseRestHandler.RequestBodyChunkConsumer { ChunkHandler(boolean allowExplicitIndex, RestRequest request, Supplier handlerSupplier) { this.request = request; this.handlerSupplier = handlerSupplier; - this.parser = new BulkRequestParser(true, RestUtils.getIncludeSourceOnError(request), request.getRestApiVersion()) - .incrementalParser( + AbstractBulkRequestParser requestParser; + if (ArrowBulkRequestParser.isArrowRequest(request)) { + requestParser = new ArrowBulkRequestParser(request); + } else { + requestParser = new BulkRequestParser(true, RestUtils.getIncludeSourceOnError(request), request.getRestApiVersion()) + ; + } + this.parser = requestParser.incrementalParser( request.param("index"), request.param("routing"), FetchSourceContext.parseFromRestRequest(request), @@ -252,7 +265,7 @@ public void streamClose() { private void shortCircuit() { shortCircuited = true; - Releasables.close(handler); + Releasables.close(parser, handler); Releasables.close(unParsedChunks); unParsedChunks.clear(); } diff --git a/server/src/main/resources/org/elasticsearch/bootstrap/security.policy b/server/src/main/resources/org/elasticsearch/bootstrap/security.policy index 55abdc84fc8fb..bcdbd450d420a 100644 --- a/server/src/main/resources/org/elasticsearch/bootstrap/security.policy +++ b/server/src/main/resources/org/elasticsearch/bootstrap/security.policy @@ -73,9 +73,27 @@ grant codeBase "${codebase.elasticsearch-simdvec}" { permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; }; +grant codebase "${codebase.jackson-databind}" { + permission java.lang.RuntimePermission "accessDeclaredMembers"; + permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; +}; + +// Arrow uses Unsafe to access memory and accesses a private field in java.nio.Buffer +// See also additional global permissions below +grant codebase "${codebase.arrow-memory-core}" { + permission java.lang.RuntimePermission "accessDeclaredMembers"; + permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; + permission java.lang.RuntimePermission "accessClassInPackage.sun.misc"; +}; + //// Everything else: grant { + // This should be covered by arrow-memory-core above, but field access to java.nio.Buffer.address + // from org.apache.arrow.memory.util.MemoryUtil isn't in a privileged section. + permission java.lang.RuntimePermission "accessDeclaredMembers"; + permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; + // needed by vendored Guice permission java.lang.RuntimePermission "accessClassInPackage.jdk.internal.vm.annotation"; diff --git a/server/src/test/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParserTests.java b/server/src/test/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParserTests.java new file mode 100644 index 0000000000000..7d611dd7bc156 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParserTests.java @@ -0,0 +1,470 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.action.bulk.arrow; + +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.complex.MapVector; +import org.apache.arrow.vector.complex.StructVector; +import org.apache.arrow.vector.dictionary.Dictionary; +import org.apache.arrow.vector.dictionary.DictionaryEncoder; +import org.apache.arrow.vector.dictionary.DictionaryProvider; +import org.apache.arrow.vector.ipc.ArrowStreamWriter; +import org.apache.arrow.vector.types.Types; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.DictionaryEncoding; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.FieldType; +import org.apache.arrow.vector.types.pojo.Schema; +import org.apache.arrow.vector.util.Text; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.arrow.Arrow; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.core.RestApiVersion; +import org.elasticsearch.search.fetch.subphase.FetchSourceContext; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xcontent.XContentType; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.function.BiConsumer; +import java.util.function.Consumer; + +import static org.hamcrest.Matchers.equalTo; + +public class ArrowBulkIncrementalParserTests extends ESTestCase { + + //----- Test Arrow batches and incremental parsing + + public void testBatchingAndChunking() throws IOException { + checkBatchingAndChunking(1, 10, false); + checkBatchingAndChunking(1, 10, true); + checkBatchingAndChunking(2, 10, false); + checkBatchingAndChunking(2, 10, true); + } + + /** Create a payload for a 1-column dataframe (int and string), given a number of batches and rows per batch */ + private void checkBatchingAndChunking(int batchCount, int rowCount, boolean incremental) throws IOException { + byte[] payload; + + // Create a dataframe with two columns: integer and string + Field intField = new Field("ints", FieldType.nullable(new ArrowType.Int(32, true)), null); + Field strField = new Field("strings", FieldType.nullable(new ArrowType.Utf8()), null); + Schema schema = new Schema(List.of(intField, strField)); + + // Create vectors and write them to a byte array + try ( + var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); + var root = VectorSchemaRoot.create(schema, allocator); + ) { + var baos = new ByteArrayOutputStream(); + IntVector intVector = (IntVector) root.getVector(0); + VarCharVector stringVector = (VarCharVector) root.getVector(1); + + try (ArrowStreamWriter writer = new ArrowStreamWriter(root, null, baos)) { + for (int batch = 0; batch < batchCount; batch++) { + intVector.allocateNew(rowCount); + stringVector.allocateNew(rowCount); + for (int row = 0; row < rowCount; row++) { + int globalRow = row + batch * rowCount; + intVector.set(row, globalRow); + stringVector.set(row, new Text("row" + globalRow)); + } + root.setRowCount(rowCount); + writer.writeBatch(); + } + } + payload = baos.toByteArray(); + } + + var operations = new ArrayList>(); + try (var parser = createParser("test", operations)) { + parse(parser, payload, incremental); + }; + + assertEquals(batchCount * rowCount, operations.size()); + + for (int i = 0; i < operations.size(); i++) { + IndexRequest operation = (IndexRequest)operations.get(i); + + assertEquals(DocWriteRequest.OpType.INDEX, operation.opType()); + assertEquals("test", operation.index()); + + assertEquals(XContentType.CBOR, operation.getContentType()); + + var map = operation.sourceAsMap(); + assertEquals(i, map.get("ints")); + assertEquals("row" + i, map.get("strings")); + } + } + + public void testInlineIdAndIndex() throws Exception { + byte[] payload; + + Field indexField = new Field("_index", FieldType.nullable(new ArrowType.Utf8()), null); + Field idField = new Field("_id", FieldType.nullable(new ArrowType.Utf8()), null); + Field intField = new Field("ints", FieldType.nullable(new ArrowType.Int(32, true)), null); + Field strField = new Field("strings", FieldType.nullable(new ArrowType.Utf8()), null); + Schema schema = new Schema(List.of(indexField, idField, intField, strField)); + + try ( + var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); + var root = VectorSchemaRoot.create(schema, allocator); + ) { + var baos = new ByteArrayOutputStream(); + VarCharVector indexVector = (VarCharVector) root.getVector(0); + VarCharVector idVector = (VarCharVector) root.getVector(1); + IntVector intVector = (IntVector) root.getVector(2); + VarCharVector stringVector = (VarCharVector) root.getVector(3); + + try (ArrowStreamWriter writer = new ArrowStreamWriter(root, null, baos)) { + indexVector.allocateNew(4); + idVector.allocateNew(4); + intVector.allocateNew(4); + stringVector.allocateNew(4); + + // No index, no id + indexVector.setNull(0); + idVector.setNull(0); + stringVector.set(0, new Text("row0")); + intVector.set(0, 0); + + // No index, id + indexVector.setNull(1); + idVector.set(1, new Text("id1")); + stringVector.set(1, new Text("row1")); + intVector.set(1, 1); + + // Index, no id + indexVector.set(2, new Text("index2")); + idVector.setNull(2); + stringVector.set(2, new Text("row2")); + intVector.set(2, 2); + + // Index & id + indexVector.set(3, new Text("index3")); + idVector.set(3, new Text("id3")); + stringVector.set(1, new Text("row3")); + intVector.set(1, 3); + + root.setRowCount(4); + writer.writeBatch(); + } + payload = baos.toByteArray(); + } + + var operations = new ArrayList>(); + try (var parser = createParser("defaultIndex", operations)) { + parse(parser, payload, false); + }; + + IndexRequest operation = (IndexRequest)operations.get(0); + assertEquals("defaultIndex", operation.index()); + assertEquals(null, operation.id()); + + operation = (IndexRequest)operations.get(1); + assertEquals("defaultIndex", operation.index()); + assertEquals("id1", operation.id()); + + operation = (IndexRequest)operations.get(2); + assertEquals("index2", operation.index()); + assertEquals(null, operation.id()); + + operation = (IndexRequest)operations.get(3); + assertEquals("index3", operation.index()); + assertEquals("id3", operation.id()); + + } + + //----- Test action decoding + + /** Action as a map of (string, string) */ + public void testActionsAsStringMap() throws Exception { + + try ( + var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); + var vector = new MapVector("action", allocator, FieldType.nullable(new ArrowType.Map(false)), null); + var parser = createParser("default-index", List.of()) + ) { + var w = vector.getWriter(); + + w.startMap(); + + // Override operation type (default is create) + w.startEntry(); + w.key().varChar().writeVarChar("op_type"); + w.value().varChar().writeVarChar("update"); + w.endEntry(); + + // Override default "default-index" index + w.startEntry(); + w.key().varChar().writeVarChar("_index"); + w.value().varChar().writeVarChar("first-index"); + w.endEntry(); + + // Set if_seq_no as a string, to test a lazy approach with a simple (string, string) map + w.startEntry(); + w.key().varChar().writeVarChar("if_seq_no"); + w.value().varChar().writeVarChar("3"); + w.endEntry(); + + w.endMap(); + + w.startMap(); + + // Override default "default-index" index + w.startEntry(); + w.key().varChar().writeVarChar("_index"); + w.value().varChar().writeVarChar("second-index"); + w.endEntry(); + + // Override operation type (default is create) + w.startEntry(); + w.key().varChar().writeVarChar("op_type"); + w.value().varChar().writeVarChar("index"); + w.endEntry(); + + // Set version as a string, to test a lazy approach with a simple (string, string) map + w.startEntry(); + w.key().varChar().writeVarChar("if_seq_no"); + w.value().varChar().writeVarChar("4"); + w.endEntry(); + + w.endMap(); + + vector.setValueCount(w.getPosition()); + // Value type is varchar + assertEquals(Types.MinorType.VARCHAR, vector.getChildrenFromFields().get(0).getChildrenFromFields().get(1).getMinorType()); + + { + var request = parser.parseAction(vector, 0, null, null); + assertEquals(DocWriteRequest.OpType.UPDATE, request.opType()); + assertEquals("first-index", request.index()); + assertEquals(3, request.ifSeqNo()); + } + + { + var request = parser.parseAction(vector, 1, null, null); + assertEquals(DocWriteRequest.OpType.INDEX, request.opType()); + assertEquals("second-index", request.index()); + assertEquals(4, request.ifSeqNo()); + } + } + } + + /** Action as a map of (string, union(string, int)) */ + public void testActionsAsUnionMap() throws Exception { + + try ( + var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); + var vector = new MapVector("action", allocator, FieldType.nullable(new ArrowType.Map(false)), null); + var parser = createParser("default-index", List.of()) + ) { + var w = vector.getWriter(); + + w.startMap(); + + // Override operation type (default is create) + w.startEntry(); + w.key().varChar().writeVarChar("op_type"); + w.value().varChar().writeVarChar("update"); + w.endEntry(); + + // Override default "default-index" index + w.startEntry(); + w.key().varChar().writeVarChar("_index"); + w.value().varChar().writeVarChar("some-index"); + w.endEntry(); + + // Set version as a number. This promotes the value field to a union type + w.startEntry(); + w.key().varChar().writeVarChar("if_seq_no"); + w.value().integer().writeInt(3); + w.endEntry(); + + w.endMap(); + + vector.setValueCount(w.getPosition()); + var request = parser.parseAction(vector, 0, null, null); + + // Value type is a union + assertEquals(Types.MinorType.UNION, vector.getChildrenFromFields().get(0).getChildrenFromFields().get(1).getMinorType()); + + assertEquals(DocWriteRequest.OpType.UPDATE, request.opType()); + assertEquals("some-index", request.index()); + assertEquals(3, request.ifSeqNo()); + } + } + + /** Action as a struct */ + public void testActionsAsStruct() throws Exception { + + try ( + var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); + var vector = new StructVector("action", allocator, FieldType.nullable(new ArrowType.Struct()), null); + var parser = createParser("default-index", List.of()) + ) { + var w = vector.getWriter(); + + w.start(); + w.varChar("op_type").writeVarChar("update"); + w.varChar("_index").writeVarChar("first-index"); + w.integer("if_seq_no").writeInt(3); + w.end(); + + w.start(); + w.varChar("op_type").writeVarChar("index"); + w.varChar("_index").writeVarChar("second-index"); + w.integer("if_seq_no").writeInt(4); + w.end(); + + vector.setValueCount(w.getPosition()); + + { + var request = parser.parseAction(vector, 0, null, null); + assertEquals(DocWriteRequest.OpType.UPDATE, request.opType()); + assertEquals("first-index", request.index()); + assertEquals(3, request.ifSeqNo()); + } + + { + var request = parser.parseAction(vector, 1, null, null); + assertEquals(DocWriteRequest.OpType.INDEX, request.opType()); + assertEquals("second-index", request.index()); + assertEquals(4, request.ifSeqNo()); + } + } + } + + //----- Dictionary encoding + public void testDictionaryEncoding() throws Exception { + + ByteArrayOutputStream payload = new ByteArrayOutputStream(); + + try ( + var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); + VarCharVector dictVector = new VarCharVector("dict", allocator); + VarCharVector vector = new VarCharVector("data_field", allocator); + DictionaryProvider.MapDictionaryProvider dictionaryProvider = new DictionaryProvider.MapDictionaryProvider(); + ) { + // create dictionary lookup vector + dictVector.allocateNewSafe(); + dictVector.setSafe(0, "aa".getBytes()); + dictVector.setSafe(1, "bb".getBytes()); + dictVector.setSafe(2, "cc".getBytes()); + dictVector.setValueCount(3); + + // create dictionary + long dictionaryId = 1L; + Dictionary dictionary = new Dictionary(dictVector, new DictionaryEncoding(dictionaryId, false, /*indexType=*/null)); + + dictionaryProvider.put(dictionary); + + // create original data vector + vector.allocateNewSafe(); + vector.setSafe(0, "bb".getBytes()); + vector.setSafe(1, "bb".getBytes()); + vector.setSafe(2, "cc".getBytes()); + vector.setSafe(3, "aa".getBytes()); + vector.setValueCount(4); + + // Encode the vector with the dictionary + IntVector encodedVector = (IntVector) DictionaryEncoder.encode(vector, dictionary); + + // create VectorSchemaRoot + List fields = List.of(encodedVector.getField()); + List vectors = List.of(encodedVector); + + try ( + VectorSchemaRoot root = new VectorSchemaRoot(fields, vectors); + ArrowStreamWriter writer = new ArrowStreamWriter(root, dictionaryProvider, payload); + ) { + // write data + writer.start(); + writer.writeBatch(); + writer.end(); + } + + var operations = new ArrayList>(); + try (var parser = createParser("defaultIndex", operations)) { + parse(parser, payload.toByteArray(), false); + }; + + // Check that dictionary-encoded values were correctly decoded + assertEquals("bb", ((IndexRequest)operations.get(0)).sourceAsMap().get("data_field")); + assertEquals("bb", ((IndexRequest)operations.get(1)).sourceAsMap().get("data_field")); + assertEquals("cc", ((IndexRequest)operations.get(2)).sourceAsMap().get("data_field")); + assertEquals("aa", ((IndexRequest)operations.get(3)).sourceAsMap().get("data_field")); + } + } + + //----- Utilities + + private static ArrowBulkIncrementalParser createParser(String defaultIndex, List> requests) { + + DocWriteRequest.OpType defaultOpType = DocWriteRequest.OpType.INDEX; + String defaultRouting = null; + FetchSourceContext defaultFetchSourceContext = null; + String defaultPipeline = null; + Boolean defaultRequireAlias = false; + Boolean defaultRequireDataStream = false; + Boolean defaultListExecutedPipelines = false; + + boolean allowExplicitIndex = true; + XContentType xContentType = null; + BiConsumer indexRequestConsumer = (r, t) -> requests.add(r); + Consumer updateRequestConsumer = requests::add; + Consumer deleteRequestConsumer = requests::add; + + return new ArrowBulkIncrementalParser( + defaultOpType, + defaultIndex, + defaultRouting, + defaultFetchSourceContext, + defaultPipeline, + defaultRequireAlias, + defaultRequireDataStream, + defaultListExecutedPipelines, + allowExplicitIndex, + xContentType, + XContentParserConfiguration.EMPTY.withRestApiVersion(RestApiVersion.current()), + indexRequestConsumer, + updateRequestConsumer, + deleteRequestConsumer + ); + } + + private void parse(ArrowBulkIncrementalParser parser, byte[] payload, boolean incremental) throws IOException { + + int consumed = 0; + var request = new BytesArray(payload); + + if (incremental) { + // Borrowed from BulkRequestParserTests + for (int i = 0; i < request.length() - 1; ++i) { + consumed += parser.parse(request.slice(consumed, i - consumed + 1), false); + } + consumed += parser.parse(request.slice(consumed, request.length() - consumed), true); + assertThat(consumed, equalTo(request.length())); + } else { + consumed = parser.parse(request, true); + } + + assertEquals(payload.length, consumed); + } +} diff --git a/server/src/test/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannelTests.java b/server/src/test/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannelTests.java new file mode 100644 index 0000000000000..f9663649656eb --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannelTests.java @@ -0,0 +1,50 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.action.bulk.arrow; + +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.bytes.CompositeBytesReference; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.nio.channels.Channels; +import java.util.ArrayList; + +public class BytesReferenceChannelTests extends ESTestCase { + + /** + * Check iteration on the buffers of a composite byteref + */ + public void testMultipleBuffers() throws IOException { + + var chunks = new ArrayList(); + byte[] chunk = null; + for (int i = 0; i < 0x100; i++) { + if (i % 0x10 == 0) { + chunk = new byte[0x10]; + chunks.add(new BytesArray(chunk)); + } + chunk[i % 0x10] = (byte)i; + } + + var bytesref = CompositeBytesReference.of(chunks.toArray(new BytesReference[0])); + + try(var channel = new BytesReferenceChannel(bytesref)) { + var in = Channels.newInputStream(channel); + + for (int i = 0; i < 0x100; i++) { + assertEquals(i, in.read()); + } + + assertEquals(-1, in.read()); + } + } +} diff --git a/x-pack/plugin/esql/arrow/build.gradle b/x-pack/plugin/esql/arrow/build.gradle index fac0bd0a77452..fface6484e125 100644 --- a/x-pack/plugin/esql/arrow/build.gradle +++ b/x-pack/plugin/esql/arrow/build.gradle @@ -12,48 +12,9 @@ dependencies { 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}" + implementation project(":libs:arrow") testImplementation project(':test:framework') - testImplementation('org.apache.arrow:arrow-memory-unsafe:16.1.0') - testImplementation("com.fasterxml.jackson.datatype:jackson-datatype-jsr310:${versions.jackson}") -} - -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' - ) } tasks.named("test").configure { diff --git a/x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/AllocationManagerShim.java b/x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/AllocationManagerShim.java deleted file mode 100644 index b52d1053ff595..0000000000000 --- a/x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/AllocationManagerShim.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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. - *

- * 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. - *

- * 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) () -> { - 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"); - } -} diff --git a/x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/ArrowResponse.java b/x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/ArrowResponse.java index 208d3308d508b..0c02e0e698a7b 100644 --- a/x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/ArrowResponse.java +++ b/x-pack/plugin/esql/arrow/src/main/java/org/elasticsearch/xpack/esql/arrow/ArrowResponse.java @@ -128,11 +128,6 @@ public void close() { * 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) { diff --git a/x-pack/plugin/esql/qa/server/single-node/build.gradle b/x-pack/plugin/esql/qa/server/single-node/build.gradle index 48880634fe348..cea6ce20a2069 100644 --- a/x-pack/plugin/esql/qa/server/single-node/build.gradle +++ b/x-pack/plugin/esql/qa/server/single-node/build.gradle @@ -9,17 +9,16 @@ dependencies { javaRestTestImplementation project(xpackModule('esql:tools')) 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(project(":libs:arrow")) + javaRestTestImplementation("org.checkerframework:checker-qual:3.48.1") + javaRestTestImplementation("com.google.flatbuffers:flatbuffers-java:24.3.25") javaRestTestImplementation("com.fasterxml.jackson.core:jackson-core:${versions.jackson}") + javaRestTestImplementation("com.fasterxml.jackson.core:jackson-annotations:${versions.jackson}") javaRestTestImplementation("com.fasterxml.jackson.core:jackson-databind:${versions.jackson}") + javaRestTestImplementation("com.fasterxml.jackson.datatype:jackson-datatype-jsr310:${versions.jackson}") + javaRestTestImplementation("commons-codec:commons-codec:${versions.commonscodec}") javaRestTestImplementation("org.slf4j:slf4j-api:${versions.slf4j}") javaRestTestImplementation("org.slf4j:slf4j-nop:${versions.slf4j}") - javaRestTestImplementation('org.apache.arrow:arrow-memory-unsafe:16.1.0') clusterPlugins project(':plugins:mapper-size') clusterPlugins project(':plugins:mapper-murmur3') diff --git a/x-pack/plugin/esql/src/main/plugin-metadata/plugin-security.codebases b/x-pack/plugin/esql/src/main/plugin-metadata/plugin-security.codebases deleted file mode 100644 index ecae5129b3563..0000000000000 --- a/x-pack/plugin/esql/src/main/plugin-metadata/plugin-security.codebases +++ /dev/null @@ -1 +0,0 @@ -arrow: org.elasticsearch.xpack.esql.arrow.AllocationManagerShim diff --git a/x-pack/plugin/esql/src/main/plugin-metadata/plugin-security.policy b/x-pack/plugin/esql/src/main/plugin-metadata/plugin-security.policy deleted file mode 100644 index 22884437add88..0000000000000 --- a/x-pack/plugin/esql/src/main/plugin-metadata/plugin-security.policy +++ /dev/null @@ -1,12 +0,0 @@ -/* - * 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"; -}; From be7cffc3c2f767afb64d6f415f2baf4b861c7d49 Mon Sep 17 00:00:00 2001 From: Sylvain Wallez Date: Mon, 17 Mar 2025 17:29:05 +0100 Subject: [PATCH 02/19] Update docs/changelog/125040.yaml --- docs/changelog/125040.yaml | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 docs/changelog/125040.yaml diff --git a/docs/changelog/125040.yaml b/docs/changelog/125040.yaml new file mode 100644 index 0000000000000..cdc1989c7285e --- /dev/null +++ b/docs/changelog/125040.yaml @@ -0,0 +1,5 @@ +pr: 125040 +summary: Add Apache Arrow as a bulk ingestion format +area: CRUD +type: enhancement +issues: [] From d23ba11527fcb879729f938707bfbfb48990b10c Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Mon, 17 Mar 2025 16:39:57 +0000 Subject: [PATCH 03/19] [CI] Auto commit changes from spotless --- .../elasticsearch/arrow/ArrowJsonParser.java | 1 + .../arrow/ArrowJsonXContentParser.java | 2 +- .../elasticsearch/arrow/ArrowToXContent.java | 83 ++++++++----------- .../elasticsearch/arrow/XContentBuffer.java | 7 +- .../arrow/ArrowToXContentTests.java | 8 +- .../EntitlementInitialization.java | 4 +- .../action/bulk/arrow/BulkArrowIT.java | 6 +- .../action/bulk/BulkRequestParser.java | 25 ++++-- .../arrow/ArrowBulkIncrementalParser.java | 56 ++++++------- .../bulk/arrow/ArrowBulkRequestParser.java | 30 +++---- .../bulk/arrow/ArrowIncrementalParser.java | 2 +- .../rest/action/document/RestBulkAction.java | 29 ++++--- .../ArrowBulkIncrementalParserTests.java | 35 ++++---- .../arrow/BytesReferenceChannelTests.java | 4 +- 14 files changed, 138 insertions(+), 154 deletions(-) diff --git a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonParser.java b/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonParser.java index 557562ea3f9f8..0487bd75ab4c1 100644 --- a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonParser.java +++ b/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonParser.java @@ -15,6 +15,7 @@ import com.fasterxml.jackson.core.JsonToken; import com.fasterxml.jackson.core.util.JsonParserDelegate; import com.fasterxml.jackson.databind.util.TokenBuffer; + import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.memory.RootAllocator; import org.apache.arrow.vector.BaseIntVector; diff --git a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonXContentParser.java b/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonXContentParser.java index f5cb63ef99292..e61d25dd5f795 100644 --- a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonXContentParser.java +++ b/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonXContentParser.java @@ -34,7 +34,7 @@ class ArrowJsonXContentParser extends AbstractXContentParser { ArrowJsonXContentParser(XContentParserConfiguration config, JsonParser parser) { super(config.registry(), config.deprecationHandler(), config.restApiVersion()); - //this.parser = ((XContentParserConfigurationImpl) config).filter(parser); + // this.parser = ((XContentParserConfigurationImpl) config).filter(parser); this.parser = parser; } diff --git a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowToXContent.java b/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowToXContent.java index 7787fc8583e66..6fc4f32b9c7a3 100644 --- a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowToXContent.java +++ b/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowToXContent.java @@ -60,9 +60,8 @@ public class ArrowToXContent { * @param dictionaries to look up values for dictionary-encoded vectors * @param generator XContent output */ - public static void writeField( - ValueVector vector, int position, Map dictionaries, XContentGenerator generator - ) throws IOException { + public static void writeField(ValueVector vector, int position, Map dictionaries, XContentGenerator generator) + throws IOException { generator.writeFieldName(vector.getName()); writeValue(vector, position, dictionaries, generator); } @@ -75,9 +74,8 @@ public static void writeField( * @param dictionaries to look up values for dictionary-encoded vectors * @param generator XContent output */ - public static void writeValue( - ValueVector vector, int position, Map dictionaries, XContentGenerator generator - ) throws IOException { + public static void writeValue(ValueVector vector, int position, Map dictionaries, XContentGenerator generator) + throws IOException { if (vector.isNull(position)) { generator.writeNull(); return; @@ -88,53 +86,53 @@ public static void writeValue( // Note: to improve performance and reduce GC thrashing, we could eagerly convert dictionary // VarCharVectors to String arrays (likely the most frequent use of dictionaries) Dictionary dictionary = dictionaries.get(dictEncoding.getId()); - position = (int) ((BaseIntVector)vector).getValueAsLong(position); + position = (int) ((BaseIntVector) vector).getValueAsLong(position); vector = dictionary.getVector(); } Void x = switch (vector.getMinorType()) { - //----- Primitive values + // ----- Primitive values case BIT -> { - generator.writeBoolean(((BitVector)vector).get(position) != 0); + generator.writeBoolean(((BitVector) vector).get(position) != 0); yield null; } case TINYINT, SMALLINT, INT, BIGINT, UINT1, UINT2, UINT4, UINT8 -> { - generator.writeNumber(((BaseIntVector)vector).getValueAsLong(position)); + generator.writeNumber(((BaseIntVector) vector).getValueAsLong(position)); yield null; } case FLOAT2, FLOAT4, FLOAT8 -> { - generator.writeNumber(((FloatingPointVector)vector).getValueAsDouble(position)); + generator.writeNumber(((FloatingPointVector) vector).getValueAsDouble(position)); yield null; } - //----- strings and bytes + // ----- strings and bytes case VARCHAR, LARGEVARCHAR, VIEWVARCHAR -> { - var bytesVector = (VariableWidthFieldVector)vector; + var bytesVector = (VariableWidthFieldVector) vector; generator.writeString(new String(bytesVector.get(position), StandardCharsets.UTF_8)); yield null; } case VARBINARY, LARGEVARBINARY, VIEWVARBINARY -> { - var bytesVector = (VariableWidthFieldVector)vector; + var bytesVector = (VariableWidthFieldVector) vector; generator.writeBinary(bytesVector.get(position)); yield null; } case FIXEDSIZEBINARY -> { - var bytesVector = (FixedSizeBinaryVector)vector; + var bytesVector = (FixedSizeBinaryVector) vector; generator.writeBinary(bytesVector.get(position)); yield null; } - //----- lists + // ----- lists case LIST, FIXED_SIZE_LIST, LISTVIEW -> { - var listVector = (BaseListVector)vector; + var listVector = (BaseListVector) vector; var valueVector = listVector.getChildrenFromFields().get(0); int start = listVector.getElementStartIndex(position); int end = listVector.getElementEndIndex(position); @@ -147,7 +145,7 @@ public static void writeValue( yield null; } - //----- Time & Timestamp (time + timezone) + // ----- Time & Timestamp (time + timezone) // Timestamps are the elapsed time since the Epoch, with an optional timezone that // can be used for timezome-aware operations or display. Since ES date fields @@ -156,44 +154,44 @@ public static void writeValue( // and https://www.elastic.co/guide/en/elasticsearch/reference/current/date.html case TIMESEC, TIMESTAMPSEC -> { - var tsVector = (TimeStampVector)vector; - generator.writeNumber(tsVector.get(position)*1000); + var tsVector = (TimeStampVector) vector; + generator.writeNumber(tsVector.get(position) * 1000); yield null; } case TIMEMILLI, TIMESTAMPMILLI -> { - var tsVector = (TimeStampVector)vector; + var tsVector = (TimeStampVector) vector; generator.writeNumber(tsVector.get(position)); yield null; } case TIMEMICRO, TIMESTAMPMICRO -> { - var tsVector = (TimeStampVector)vector; - generator.writeNumber(tsVector.get(position)/1000); + var tsVector = (TimeStampVector) vector; + generator.writeNumber(tsVector.get(position) / 1000); yield null; } case TIMENANO, TIMESTAMPNANO -> { - var tsVector = (TimeStampVector)vector; - generator.writeNumber(tsVector.get(position)/1_000_000); + var tsVector = (TimeStampVector) vector; + generator.writeNumber(tsVector.get(position) / 1_000_000); yield null; } - //----- Composite types + // ----- Composite types case MAP -> { // A map is a container vector composed of a list of struct values with "key" and "value" fields. The MapVector // is nullable, but if a map is set at a given index, there must be an entry. In other words, the StructVector data is // non-nullable. Also for a given entry, the "key" is non-nullable, however the "value" can be null. - var mapVector = (MapVector)vector; - var structVector = (StructVector)mapVector.getChildrenFromFields().get(0); + var mapVector = (MapVector) vector; + var structVector = (StructVector) mapVector.getChildrenFromFields().get(0); var kVector = structVector.getChildrenFromFields().get(0); if (STRING_TYPES.contains(kVector.getMinorType()) == false) { throw new ArrowFormatException("Arrow maps must have string keys to be converted to JSON"); } - var keyVector = (VarCharVector)kVector; + var keyVector = (VarCharVector) kVector; var valueVector = structVector.getChildrenFromFields().get(1); int start = mapVector.getElementStartIndex(position); @@ -213,9 +211,9 @@ public static void writeValue( } case STRUCT -> { - var structVector = (StructVector)vector; + var structVector = (StructVector) vector; generator.writeStartObject(); - for (var field: structVector.getChildrenFromFields()) { + for (var field : structVector.getChildrenFromFields()) { generator.writeFieldName(field.getName()); writeValue(field, position, dictionaries, generator); } @@ -224,7 +222,7 @@ public static void writeValue( } case DENSEUNION -> { - var unionVector = (DenseUnionVector)vector; + var unionVector = (DenseUnionVector) vector; var typeId = unionVector.getTypeId(position); var valueVector = unionVector.getVectorByType(typeId); var valuePosition = unionVector.getOffset(position); @@ -234,7 +232,7 @@ public static void writeValue( } case UNION -> { // sparse union - var unionVector = (UnionVector)vector; + var unionVector = (UnionVector) vector; var typeId = unionVector.getTypeValue(position); var valueVector = unionVector.getVectorByType(typeId); @@ -248,22 +246,9 @@ public static void writeValue( } // TODO - case DATEDAY, - DATEMILLI, - INTERVALDAY, - INTERVALMONTHDAYNANO, - DURATION, - INTERVALYEAR, - DECIMAL, - DECIMAL256, - LARGELIST, - LARGELISTVIEW, - TIMESTAMPSECTZ, - TIMESTAMPMILLITZ, - TIMESTAMPMICROTZ, - TIMESTAMPNANOTZ, - EXTENSIONTYPE, - RUNENDENCODED -> throw new ArrowFormatException( + case DATEDAY, DATEMILLI, INTERVALDAY, INTERVALMONTHDAYNANO, DURATION, INTERVALYEAR, DECIMAL, DECIMAL256, LARGELIST, + LARGELISTVIEW, TIMESTAMPSECTZ, TIMESTAMPMILLITZ, TIMESTAMPMICROTZ, TIMESTAMPNANOTZ, EXTENSIONTYPE, RUNENDENCODED -> + throw new ArrowFormatException( "Arrow type [" + vector.getMinorType() + "] not supported for field [" + vector.getName() + "]" ); }; diff --git a/libs/arrow/src/main/java/org/elasticsearch/arrow/XContentBuffer.java b/libs/arrow/src/main/java/org/elasticsearch/arrow/XContentBuffer.java index 2970c369c6505..16f00a485b29d 100644 --- a/libs/arrow/src/main/java/org/elasticsearch/arrow/XContentBuffer.java +++ b/libs/arrow/src/main/java/org/elasticsearch/arrow/XContentBuffer.java @@ -10,7 +10,6 @@ package org.elasticsearch.arrow; import com.fasterxml.jackson.core.JsonGenerationException; - import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.util.TokenBuffer; @@ -56,8 +55,7 @@ public XContentType contentType() { } @Override - public final void usePrettyPrint() { - } + public final void usePrettyPrint() {} @Override public boolean isPrettyPrint() { @@ -65,8 +63,7 @@ public boolean isPrettyPrint() { } @Override - public void usePrintLineFeedAtEnd() { - } + public void usePrintLineFeedAtEnd() {} @Override public void writeStartObject() throws IOException { diff --git a/libs/arrow/src/test/java/org/elasticsearch/arrow/ArrowToXContentTests.java b/libs/arrow/src/test/java/org/elasticsearch/arrow/ArrowToXContentTests.java index 352684dc7240d..3207ff1cd9efa 100644 --- a/libs/arrow/src/test/java/org/elasticsearch/arrow/ArrowToXContentTests.java +++ b/libs/arrow/src/test/java/org/elasticsearch/arrow/ArrowToXContentTests.java @@ -25,9 +25,7 @@ public class ArrowToXContentTests extends ESTestCase { private static void checkPosition(ValueVector vector, int position, String json) throws IOException { var out = new ByteArrayOutputStream(); - try ( - var generator = XContentType.JSON.xContent().createGenerator(out) - ) { + try (var generator = XContentType.JSON.xContent().createGenerator(out)) { generator.writeStartObject(); ArrowToXContent.writeField(vector, position, null, generator); generator.writeEndObject(); @@ -95,9 +93,7 @@ public void testWriteNullValue() throws Exception { } public void testWriteNullVector() throws Exception { - try ( - NullVector vector = new NullVector("nullField", 1); - ) { + try (NullVector vector = new NullVector("nullField", 1);) { checkPosition(vector, 0, "{\"nullField\":null}"); } } diff --git a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java index 984e628c84b1c..94a8629f7f619 100644 --- a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java +++ b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java @@ -276,9 +276,7 @@ private static PolicyManager createPolicyManager() { serverScopes, new Scope( "com.fasterxml.jackson.core", - List.of( - new FilesEntitlement(List.of(FileData.ofPath(bootstrapArgs.modulesDir(), READ))) - ) + List.of(new FilesEntitlement(List.of(FileData.ofPath(bootstrapArgs.modulesDir(), READ)))) ) ); diff --git a/server/src/javaRestTest/java/org/elasticsearch/action/bulk/arrow/BulkArrowIT.java b/server/src/javaRestTest/java/org/elasticsearch/action/bulk/arrow/BulkArrowIT.java index 8f46bde3945c0..0580ac66796e5 100644 --- a/server/src/javaRestTest/java/org/elasticsearch/action/bulk/arrow/BulkArrowIT.java +++ b/server/src/javaRestTest/java/org/elasticsearch/action/bulk/arrow/BulkArrowIT.java @@ -99,7 +99,7 @@ public void testBulk() throws Exception { .map(); assertEquals(Boolean.FALSE, result.get("errors")); - assertEquals(batchCount * rowCount, ((List)result.get("items")).size()); + assertEquals(batchCount * rowCount, ((List) result.get("items")).size()); } { @@ -116,7 +116,7 @@ public void testBulk() throws Exception { public void testDictionary() throws Exception { - //DictionaryEncoding encoding = new DictionaryEncoding() + // DictionaryEncoding encoding = new DictionaryEncoding() // Create a dataframe with two columns: integer and string Field intField = new Field("ints", FieldType.nullable(new ArrowType.Int(32, true)), null); @@ -165,7 +165,7 @@ public void testDictionary() throws Exception { .map(); assertEquals(Boolean.FALSE, result.get("errors")); - assertEquals(batchCount * rowCount, ((List)result.get("items")).size()); + assertEquals(batchCount * rowCount, ((List) result.get("items")).size()); } { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java index 27f76d9a8ec29..ed4e0e2de051f 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java @@ -509,7 +509,8 @@ protected DocWriteRequest parseActionLine(XContentParser parser) throws IOExc .setRequireAlias(requireAlias) .setRequireDataStream(requireDataStream) .setListExecutedPipelines(currentListExecutedPipelines) - .setIncludeSourceOnError(config.includeSourceOnError());if ("create".equals(action)) { + .setIncludeSourceOnError(config.includeSourceOnError()); + if ("create".equals(action)) { indexRequest = indexRequest.create(true); } else if (opType != null) { indexRequest = indexRequest.create("create".equals(opType)); @@ -576,9 +577,8 @@ private void parseAndConsumeDocumentLine(BytesReference data, int from, int to) } @UpdateForV10(owner = UpdateForV10.Owner.DATA_MANAGEMENT) // Remove lenient parsing in V8 BWC mode - private static void checkBulkActionIsProperlyClosed( - XContentParser parser, XContentParserConfiguration config, int line - ) throws IOException { + private static void checkBulkActionIsProperlyClosed(XContentParser parser, XContentParserConfiguration config, int line) + throws IOException { XContentParser.Token token; try { token = parser.nextToken(); @@ -629,9 +629,8 @@ private static void checkBulkActionIsProperlyClosed( } } - private static XContentParser createParser( - XContent xContent, XContentParserConfiguration config, BytesReference data - ) throws IOException { + private static XContentParser createParser(XContent xContent, XContentParserConfiguration config, BytesReference data) + throws IOException { if (data.hasArray()) { return parseBytesArray(xContent, config, data, 0, data.length()); } else { @@ -642,7 +641,11 @@ private static XContentParser createParser( // Create an efficient parser of the given bytes, trying to directly parse a byte array if possible and falling back to stream wrapping // otherwise. private static XContentParser createParser( - XContent xContent, XContentParserConfiguration config, BytesReference data, int from, int nextMarker + XContent xContent, + XContentParserConfiguration config, + BytesReference data, + int from, + int nextMarker ) throws IOException { if (data.hasArray()) { return parseBytesArray(xContent, config, data, from, nextMarker); @@ -658,7 +661,11 @@ private static XContentParser createParser( } private static XContentParser parseBytesArray( - XContent xContent, XContentParserConfiguration config, BytesReference array, int from, int nextMarker + XContent xContent, + XContentParserConfiguration config, + BytesReference array, + int from, + int nextMarker ) throws IOException { assert array.hasArray(); final int offset = array.arrayOffset(); diff --git a/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParser.java b/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParser.java index dbb6101319af0..6651803fe50d3 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParser.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParser.java @@ -49,7 +49,6 @@ import java.util.function.BiConsumer; import java.util.function.Consumer; - class ArrowBulkIncrementalParser extends BulkRequestParser.XContentIncrementalParser { /** XContent format used to encode source documents */ @@ -109,25 +108,22 @@ class ArrowBulkIncrementalParser extends BulkRequestParser.XContentIncrementalPa // FIXME: hard-coded limit to 100 MiB per record batch. Should we add an AllocationListener that calls ES memory management? this.allocator = Arrow.rootAllocator().newChildAllocator("bulk-ingestion", 0, 100 * 1024 * 1024); - this.arrowParser = new ArrowIncrementalParser( - new RootAllocator(), - new ArrowIncrementalParser.Listener() { - @Override - public void startStream(VectorSchemaRoot schemaRoot) throws IOException { - startArrowStream(schemaRoot); - } + this.arrowParser = new ArrowIncrementalParser(new RootAllocator(), new ArrowIncrementalParser.Listener() { + @Override + public void startStream(VectorSchemaRoot schemaRoot) throws IOException { + startArrowStream(schemaRoot); + } - @Override - public void nextBatch(Map dictionary) throws IOException { - nextArrowBatch(dictionary); - } + @Override + public void nextBatch(Map dictionary) throws IOException { + nextArrowBatch(dictionary); + } - @Override - public void endStream() throws IOException { - endArrowStream(); - } + @Override + public void endStream() throws IOException { + endArrowStream(); } - ); + }); } @Override @@ -197,14 +193,15 @@ private void nextArrowBatch(Map dictionary) throws IOException case DeleteRequest dr -> { deleteRequestConsumer.accept(dr); } - default -> {} + default -> { + } } } } protected BytesReference generateSource(int position) throws IOException { var output = new BytesReferenceOutputStream(); - try(var generator = SOURCE_XCONTENT.createGenerator(output)) { + try (var generator = SOURCE_XCONTENT.createGenerator(output)) { generator.writeStartObject(); int rowCount = schemaRoot.getRowCount(); for (int i = 0; i < rowCount; i++) { @@ -286,13 +283,13 @@ private String getNamedString(FieldVector vector, String name, int position) { if (vector instanceof MapVector mapVector) { // A Map is a variable-size list of structs with two fields, key and value (in this order) var data = mapVector.getDataVector(); - var keyVec = (VarCharVector)data.getChildrenFromFields().get(0); + var keyVec = (VarCharVector) data.getChildrenFromFields().get(0); var valueVec = data.getChildrenFromFields().get(1); var key = new Text(); for (int pos = mapVector.getElementStartIndex(position); pos < mapVector.getElementEndIndex(position); pos++) { keyVec.read(pos, key); - if (Arrays.equals(nameBytes, 0, nameBytes.length, key.getBytes(), 0, (int)key.getLength())) { + if (Arrays.equals(nameBytes, 0, nameBytes.length, key.getBytes(), 0, (int) key.getLength())) { return getString(valueVec, pos); } } @@ -305,7 +302,7 @@ private String getNamedString(FieldVector vector, String name, int position) { return childVector == null ? null : getString(childVector, position); } - for (var child: vector.getChildrenFromFields()) { + for (var child : vector.getChildrenFromFields()) { if (child instanceof ValueVector valueVector && valueVector.getName().equals(name)) { return getString(valueVector, position); } @@ -319,18 +316,19 @@ private String getString(ValueVector vector, int position) { } return switch (vector.getMinorType()) { - case TINYINT, SMALLINT, INT, BIGINT, UINT1, UINT2, UINT4, UINT8 -> - String.valueOf(((BaseIntVector)vector).getValueAsLong(position)); + case TINYINT, SMALLINT, INT, BIGINT, UINT1, UINT2, UINT4, UINT8 -> String.valueOf( + ((BaseIntVector) vector).getValueAsLong(position) + ); case VARCHAR, LARGEVARCHAR, VIEWVARCHAR -> { - var bytesVector = (VariableWidthFieldVector)vector; + var bytesVector = (VariableWidthFieldVector) vector; yield new String(bytesVector.get(position), StandardCharsets.UTF_8); } case UNION -> { - UnionVector unionVector = (UnionVector)vector; + UnionVector unionVector = (UnionVector) vector; // Find the child field that isn't null, which is the active variant. - for (var variantVec: unionVector.getChildrenFromFields()) { + for (var variantVec : unionVector.getChildrenFromFields()) { if (variantVec.isNull(position) == false) { yield getString(variantVec, position); } @@ -339,9 +337,7 @@ private String getString(ValueVector vector, int position) { } default -> { - throw new ArrowFormatException( - "Arrow type [" + vector.getMinorType() + "] cannot be converted to string" - ); + throw new ArrowFormatException("Arrow type [" + vector.getMinorType() + "] cannot be converted to string"); } }; } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkRequestParser.java b/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkRequestParser.java index 5f39bda2aaf64..ed3e20b79e074 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkRequestParser.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkRequestParser.java @@ -70,20 +70,22 @@ public void parse( Consumer updateRequestConsumer, Consumer deleteRequestConsumer ) throws IOException { - try(IncrementalParser parser = incrementalParser( - defaultIndex, - defaultRouting, - defaultFetchSourceContext, - defaultPipeline, - defaultRequireAlias, - defaultRequireDataStream, - defaultListExecutedPipelines, - allowExplicitIndex, - xContentType, - indexRequestConsumer, - updateRequestConsumer, - deleteRequestConsumer - )) { + try ( + IncrementalParser parser = incrementalParser( + defaultIndex, + defaultRouting, + defaultFetchSourceContext, + defaultPipeline, + defaultRequireAlias, + defaultRequireDataStream, + defaultListExecutedPipelines, + allowExplicitIndex, + xContentType, + indexRequestConsumer, + updateRequestConsumer, + deleteRequestConsumer + ) + ) { parser.parse(data, true); } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowIncrementalParser.java b/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowIncrementalParser.java index d3d6b8a7ec48d..81a9d73c958eb 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowIncrementalParser.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowIncrementalParser.java @@ -154,6 +154,6 @@ private int doParse(BytesReference data, boolean lastData) throws IOException { } } - return (int)(reader.bytesRead() - initialBytesRead); + return (int) (reader.bytesRead() - initialBytesRead); } } diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java index f857479bec053..dbb6156d2b342 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java @@ -172,23 +172,22 @@ static class ChunkHandler implements BaseRestHandler.RequestBodyChunkConsumer { if (ArrowBulkRequestParser.isArrowRequest(request)) { requestParser = new ArrowBulkRequestParser(request); } else { - requestParser = new BulkRequestParser(true, RestUtils.getIncludeSourceOnError(request), request.getRestApiVersion()) - ; + requestParser = new BulkRequestParser(true, RestUtils.getIncludeSourceOnError(request), request.getRestApiVersion()); } this.parser = requestParser.incrementalParser( - request.param("index"), - request.param("routing"), - FetchSourceContext.parseFromRestRequest(request), - request.param("pipeline"), - request.paramAsBoolean(DocWriteRequest.REQUIRE_ALIAS, false), - request.paramAsBoolean(DocWriteRequest.REQUIRE_DATA_STREAM, false), - request.paramAsBoolean("list_executed_pipelines", false), - allowExplicitIndex, - request.getXContentType(), - (indexRequest, type) -> items.add(indexRequest), - items::add, - items::add - ); + request.param("index"), + request.param("routing"), + FetchSourceContext.parseFromRestRequest(request), + request.param("pipeline"), + request.paramAsBoolean(DocWriteRequest.REQUIRE_ALIAS, false), + request.paramAsBoolean(DocWriteRequest.REQUIRE_DATA_STREAM, false), + request.paramAsBoolean("list_executed_pipelines", false), + allowExplicitIndex, + request.getXContentType(), + (indexRequest, type) -> items.add(indexRequest), + items::add, + items::add + ); } @Override diff --git a/server/src/test/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParserTests.java b/server/src/test/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParserTests.java index 7d611dd7bc156..c29b34d7e00ae 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParserTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParserTests.java @@ -49,7 +49,7 @@ public class ArrowBulkIncrementalParserTests extends ESTestCase { - //----- Test Arrow batches and incremental parsing + // ----- Test Arrow batches and incremental parsing public void testBatchingAndChunking() throws IOException { checkBatchingAndChunking(1, 10, false); @@ -95,12 +95,13 @@ private void checkBatchingAndChunking(int batchCount, int rowCount, boolean incr var operations = new ArrayList>(); try (var parser = createParser("test", operations)) { parse(parser, payload, incremental); - }; + } + ; assertEquals(batchCount * rowCount, operations.size()); for (int i = 0; i < operations.size(); i++) { - IndexRequest operation = (IndexRequest)operations.get(i); + IndexRequest operation = (IndexRequest) operations.get(i); assertEquals(DocWriteRequest.OpType.INDEX, operation.opType()); assertEquals("test", operation.index()); @@ -171,27 +172,28 @@ public void testInlineIdAndIndex() throws Exception { var operations = new ArrayList>(); try (var parser = createParser("defaultIndex", operations)) { parse(parser, payload, false); - }; + } + ; - IndexRequest operation = (IndexRequest)operations.get(0); + IndexRequest operation = (IndexRequest) operations.get(0); assertEquals("defaultIndex", operation.index()); assertEquals(null, operation.id()); - operation = (IndexRequest)operations.get(1); + operation = (IndexRequest) operations.get(1); assertEquals("defaultIndex", operation.index()); assertEquals("id1", operation.id()); - operation = (IndexRequest)operations.get(2); + operation = (IndexRequest) operations.get(2); assertEquals("index2", operation.index()); assertEquals(null, operation.id()); - operation = (IndexRequest)operations.get(3); + operation = (IndexRequest) operations.get(3); assertEquals("index3", operation.index()); assertEquals("id3", operation.id()); } - //----- Test action decoding + // ----- Test action decoding /** Action as a map of (string, string) */ public void testActionsAsStringMap() throws Exception { @@ -351,7 +353,7 @@ public void testActionsAsStruct() throws Exception { } } - //----- Dictionary encoding + // ----- Dictionary encoding public void testDictionaryEncoding() throws Exception { ByteArrayOutputStream payload = new ByteArrayOutputStream(); @@ -403,17 +405,18 @@ public void testDictionaryEncoding() throws Exception { var operations = new ArrayList>(); try (var parser = createParser("defaultIndex", operations)) { parse(parser, payload.toByteArray(), false); - }; + } + ; // Check that dictionary-encoded values were correctly decoded - assertEquals("bb", ((IndexRequest)operations.get(0)).sourceAsMap().get("data_field")); - assertEquals("bb", ((IndexRequest)operations.get(1)).sourceAsMap().get("data_field")); - assertEquals("cc", ((IndexRequest)operations.get(2)).sourceAsMap().get("data_field")); - assertEquals("aa", ((IndexRequest)operations.get(3)).sourceAsMap().get("data_field")); + assertEquals("bb", ((IndexRequest) operations.get(0)).sourceAsMap().get("data_field")); + assertEquals("bb", ((IndexRequest) operations.get(1)).sourceAsMap().get("data_field")); + assertEquals("cc", ((IndexRequest) operations.get(2)).sourceAsMap().get("data_field")); + assertEquals("aa", ((IndexRequest) operations.get(3)).sourceAsMap().get("data_field")); } } - //----- Utilities + // ----- Utilities private static ArrowBulkIncrementalParser createParser(String defaultIndex, List> requests) { diff --git a/server/src/test/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannelTests.java b/server/src/test/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannelTests.java index f9663649656eb..1771c12e3bea0 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannelTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannelTests.java @@ -32,12 +32,12 @@ public void testMultipleBuffers() throws IOException { chunk = new byte[0x10]; chunks.add(new BytesArray(chunk)); } - chunk[i % 0x10] = (byte)i; + chunk[i % 0x10] = (byte) i; } var bytesref = CompositeBytesReference.of(chunks.toArray(new BytesReference[0])); - try(var channel = new BytesReferenceChannel(bytesref)) { + try (var channel = new BytesReferenceChannel(bytesref)) { var in = Channels.newInputStream(channel); for (int i = 0; i < 0x100; i++) { From 5dfdbd82955af11ce01e2926aac388dbe8ed1d1e Mon Sep 17 00:00:00 2001 From: Sylvain Wallez Date: Mon, 17 Mar 2025 20:03:44 +0100 Subject: [PATCH 04/19] useDefaultDistribution is not needed anymore --- server/build.gradle | 4 ---- 1 file changed, 4 deletions(-) diff --git a/server/build.gradle b/server/build.gradle index f995c297a7726..70e49abb306bf 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -291,7 +291,3 @@ tasks.named("licenseHeaders").configure { } tasks.withType(Checkstyle.class).configureEach { t -> t.getMaxHeapSize().set("2g") } - -tasks.named('javaRestTest') { - usesDefaultDistribution() -} From e1af88292e319c18d79907bbfaf9cfd749ec7727 Mon Sep 17 00:00:00 2001 From: Sylvain Wallez Date: Wed, 19 Mar 2025 14:33:51 +0100 Subject: [PATCH 05/19] Fix Arrow lib license files --- libs/arrow/build.gradle | 9 +--- libs/arrow/licenses/build.gradle | 49 ------------------- ...-LICENSE.txt => commons-codec-LICENSE.txt} | 0 ...ow-NOTICE.txt => commons-codec-NOTICE.txt} | 0 ...lf4j-LICENSE.txt => slf4j-api-LICENSE.txt} | 0 ...{slf4j-NOTICE.txt => slf4j-api-NOTICE.txt} | 0 6 files changed, 1 insertion(+), 57 deletions(-) delete mode 100644 libs/arrow/licenses/build.gradle rename libs/arrow/licenses/{commons-codec-arrow-LICENSE.txt => commons-codec-LICENSE.txt} (100%) rename libs/arrow/licenses/{commons-codec-arrow-NOTICE.txt => commons-codec-NOTICE.txt} (100%) rename libs/arrow/licenses/{slf4j-LICENSE.txt => slf4j-api-LICENSE.txt} (100%) rename libs/arrow/licenses/{slf4j-NOTICE.txt => slf4j-api-NOTICE.txt} (100%) diff --git a/libs/arrow/build.gradle b/libs/arrow/build.gradle index 1c9676131ad19..156fca8dd84b1 100644 --- a/libs/arrow/build.gradle +++ b/libs/arrow/build.gradle @@ -38,7 +38,7 @@ dependencies { api("com.google.flatbuffers:flatbuffers-java:24.3.25") api("commons-codec:commons-codec:${versions.commonscodec}") // Arrow 18 -> commons-codec 1.17.1 - api("org.slf4j:slf4j-api:${versions.slf4j}") + api("org.slf4j:slf4j-api:2.0.10") api("org.immutables:value-annotations:2.10.1") // provided dependency // arrow-format @@ -63,11 +63,7 @@ dependencies { tasks.named("dependencyLicenses").configure { mapping from: /jackson-.*/, to: 'jackson' mapping from: /arrow-.*/, to: 'arrow' - mapping from: /slf4j-.*/, to: 'slf4j' mapping from: /value-annotations.*/, to: 'org-immutables' - - // FIXME: need dependency above, but build complains about duplicated license files. - mapping from: /commons-codec/, to: 'commons-codec-arrow' } tasks.named("thirdPartyAudit").configure { @@ -85,7 +81,4 @@ tasks.named("thirdPartyAudit").configure { 'org.apache.arrow.vector.BitVectorHelper', 'org.apache.arrow.memory.util.ByteFunctionHelpers', ) - ignoreMissingClasses( - 'org.apache.commons.codec.binary.Hex' - ) } diff --git a/libs/arrow/licenses/build.gradle b/libs/arrow/licenses/build.gradle deleted file mode 100644 index 0fe358c4b3703..0000000000000 --- a/libs/arrow/licenses/build.gradle +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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. - */ - -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 project(":libs:arrow") - testImplementation project(':test:framework') -} - -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' - ) -} - -tasks.named("test").configure { - jvmArgs('--add-opens=java.base/java.nio=ALL-UNNAMED') -} diff --git a/libs/arrow/licenses/commons-codec-arrow-LICENSE.txt b/libs/arrow/licenses/commons-codec-LICENSE.txt similarity index 100% rename from libs/arrow/licenses/commons-codec-arrow-LICENSE.txt rename to libs/arrow/licenses/commons-codec-LICENSE.txt diff --git a/libs/arrow/licenses/commons-codec-arrow-NOTICE.txt b/libs/arrow/licenses/commons-codec-NOTICE.txt similarity index 100% rename from libs/arrow/licenses/commons-codec-arrow-NOTICE.txt rename to libs/arrow/licenses/commons-codec-NOTICE.txt diff --git a/libs/arrow/licenses/slf4j-LICENSE.txt b/libs/arrow/licenses/slf4j-api-LICENSE.txt similarity index 100% rename from libs/arrow/licenses/slf4j-LICENSE.txt rename to libs/arrow/licenses/slf4j-api-LICENSE.txt diff --git a/libs/arrow/licenses/slf4j-NOTICE.txt b/libs/arrow/licenses/slf4j-api-NOTICE.txt similarity index 100% rename from libs/arrow/licenses/slf4j-NOTICE.txt rename to libs/arrow/licenses/slf4j-api-NOTICE.txt From 604da53e59b6a8f4e52c8e760da22eb5ec8fd9fc Mon Sep 17 00:00:00 2001 From: Sylvain Wallez Date: Wed, 19 Mar 2025 14:34:33 +0100 Subject: [PATCH 06/19] Remove licence files now provided by the server module via libs:arrow --- .../licenses/commons-codec-LICENSE.txt | 202 ------------------ .../licenses/commons-codec-NOTICE.txt | 17 -- .../licenses/slf4j-api-LICENSE.txt | 21 -- .../licenses/slf4j-api-NOTICE.txt | 0 .../ingest-geoip/licenses/jackson-LICENSE.txt | 8 - .../ingest-geoip/licenses/jackson-NOTICE.txt | 20 -- .../legacy-geo/licenses/jackson-LICENSE.txt | 8 - .../legacy-geo/licenses/jackson-NOTICE.txt | 20 -- .../licenses/slf4j-api-LICENSE.txt | 21 -- .../licenses/slf4j-api-NOTICE.txt | 0 .../licenses/commons-codec-LICENSE.txt | 202 ------------------ .../licenses/commons-codec-NOTICE.txt | 17 -- .../licenses/jackson-LICENSE.txt | 8 - .../licenses/jackson-NOTICE.txt | 20 -- .../licenses/commons-codec-LICENSE.txt | 202 ------------------ .../licenses/commons-codec-NOTICE.txt | 17 -- .../repository-s3/licenses/jackson-LICENSE | 8 - modules/repository-s3/licenses/jackson-NOTICE | 20 -- .../licenses/commons-codec-LICENSE.txt | 202 ------------------ .../licenses/commons-codec-NOTICE.txt | 17 -- .../licenses/commons-codec-LICENSE.txt | 202 ------------------ .../licenses/commons-codec-NOTICE.txt | 5 - .../licenses/commons-codec-LICENSE.txt | 202 ------------------ .../licenses/commons-codec-NOTICE.txt | 17 -- .../licenses/commons-codec-LICENSE.txt | 202 ------------------ .../licenses/commons-codec-NOTICE.txt | 17 -- .../discovery-ec2/licenses/jackson-LICENSE | 8 - plugins/discovery-ec2/licenses/jackson-NOTICE | 20 -- .../licenses/commons-codec-LICENSE.txt | 202 ------------------ .../licenses/commons-codec-NOTICE.txt | 17 -- .../licenses/jackson-LICENSE.txt | 8 - .../discovery-gce/licenses/jackson-NOTICE.txt | 20 -- .../licenses/commons-codec-LICENSE.txt | 202 ------------------ .../licenses/commons-codec-NOTICE.txt | 17 -- .../licenses/slf4j-api-LICENSE.txt | 21 -- .../licenses/slf4j-api-NOTICE.txt | 0 .../ent-search/licenses/jackson-LICENSE | 8 - .../plugin/ent-search/licenses/jackson-NOTICE | 20 -- .../ent-search/licenses/slf4j-api-LICENSE.txt | 21 -- .../ent-search/licenses/slf4j-api-NOTICE.txt | 0 .../licenses/slf4j-api-LICENSE.txt | 21 -- .../licenses/slf4j-api-NOTICE.txt | 0 .../inference/licenses/jackson-LICENSE.txt | 8 - .../inference/licenses/jackson-NOTICE.txt | 20 -- .../security/licenses/slf4j-api-LICENSE.txt | 21 -- .../security/licenses/slf4j-api-NOTICE.txt | 0 .../licenses/slf4j-api-LICENSE.txt | 21 -- .../vector-tile/licenses/slf4j-api-NOTICE.txt | 0 48 files changed, 2330 deletions(-) delete mode 100644 modules/ingest-attachment/licenses/commons-codec-LICENSE.txt delete mode 100644 modules/ingest-attachment/licenses/commons-codec-NOTICE.txt delete mode 100644 modules/ingest-attachment/licenses/slf4j-api-LICENSE.txt delete mode 100644 modules/ingest-attachment/licenses/slf4j-api-NOTICE.txt delete mode 100644 modules/ingest-geoip/licenses/jackson-LICENSE.txt delete mode 100644 modules/ingest-geoip/licenses/jackson-NOTICE.txt delete mode 100644 modules/legacy-geo/licenses/jackson-LICENSE.txt delete mode 100644 modules/legacy-geo/licenses/jackson-NOTICE.txt delete mode 100644 modules/repository-azure/licenses/slf4j-api-LICENSE.txt delete mode 100644 modules/repository-azure/licenses/slf4j-api-NOTICE.txt delete mode 100644 modules/repository-gcs/licenses/commons-codec-LICENSE.txt delete mode 100644 modules/repository-gcs/licenses/commons-codec-NOTICE.txt delete mode 100644 modules/repository-gcs/licenses/jackson-LICENSE.txt delete mode 100644 modules/repository-gcs/licenses/jackson-NOTICE.txt delete mode 100644 modules/repository-s3/licenses/commons-codec-LICENSE.txt delete mode 100644 modules/repository-s3/licenses/commons-codec-NOTICE.txt delete mode 100644 modules/repository-s3/licenses/jackson-LICENSE delete mode 100644 modules/repository-s3/licenses/jackson-NOTICE delete mode 100644 modules/repository-url/licenses/commons-codec-LICENSE.txt delete mode 100644 modules/repository-url/licenses/commons-codec-NOTICE.txt delete mode 100644 plugins/analysis-phonetic/licenses/commons-codec-LICENSE.txt delete mode 100644 plugins/analysis-phonetic/licenses/commons-codec-NOTICE.txt delete mode 100644 plugins/discovery-azure-classic/licenses/commons-codec-LICENSE.txt delete mode 100644 plugins/discovery-azure-classic/licenses/commons-codec-NOTICE.txt delete mode 100644 plugins/discovery-ec2/licenses/commons-codec-LICENSE.txt delete mode 100644 plugins/discovery-ec2/licenses/commons-codec-NOTICE.txt delete mode 100644 plugins/discovery-ec2/licenses/jackson-LICENSE delete mode 100644 plugins/discovery-ec2/licenses/jackson-NOTICE delete mode 100644 plugins/discovery-gce/licenses/commons-codec-LICENSE.txt delete mode 100644 plugins/discovery-gce/licenses/commons-codec-NOTICE.txt delete mode 100644 plugins/discovery-gce/licenses/jackson-LICENSE.txt delete mode 100644 plugins/discovery-gce/licenses/jackson-NOTICE.txt delete mode 100644 plugins/repository-hdfs/licenses/commons-codec-LICENSE.txt delete mode 100644 plugins/repository-hdfs/licenses/commons-codec-NOTICE.txt delete mode 100644 plugins/repository-hdfs/licenses/slf4j-api-LICENSE.txt delete mode 100644 plugins/repository-hdfs/licenses/slf4j-api-NOTICE.txt delete mode 100644 x-pack/plugin/ent-search/licenses/jackson-LICENSE delete mode 100644 x-pack/plugin/ent-search/licenses/jackson-NOTICE delete mode 100644 x-pack/plugin/ent-search/licenses/slf4j-api-LICENSE.txt delete mode 100644 x-pack/plugin/ent-search/licenses/slf4j-api-NOTICE.txt delete mode 100644 x-pack/plugin/identity-provider/licenses/slf4j-api-LICENSE.txt delete mode 100644 x-pack/plugin/identity-provider/licenses/slf4j-api-NOTICE.txt delete mode 100644 x-pack/plugin/inference/licenses/jackson-LICENSE.txt delete mode 100644 x-pack/plugin/inference/licenses/jackson-NOTICE.txt delete mode 100644 x-pack/plugin/security/licenses/slf4j-api-LICENSE.txt delete mode 100644 x-pack/plugin/security/licenses/slf4j-api-NOTICE.txt delete mode 100644 x-pack/plugin/vector-tile/licenses/slf4j-api-LICENSE.txt delete mode 100644 x-pack/plugin/vector-tile/licenses/slf4j-api-NOTICE.txt diff --git a/modules/ingest-attachment/licenses/commons-codec-LICENSE.txt b/modules/ingest-attachment/licenses/commons-codec-LICENSE.txt deleted file mode 100644 index 57bc88a15a0ee..0000000000000 --- a/modules/ingest-attachment/licenses/commons-codec-LICENSE.txt +++ /dev/null @@ -1,202 +0,0 @@ - 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. - diff --git a/modules/ingest-attachment/licenses/commons-codec-NOTICE.txt b/modules/ingest-attachment/licenses/commons-codec-NOTICE.txt deleted file mode 100644 index 56916449bbe10..0000000000000 --- a/modules/ingest-attachment/licenses/commons-codec-NOTICE.txt +++ /dev/null @@ -1,17 +0,0 @@ -Apache Commons Codec -Copyright 2002-2015 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java -contains test data from http://aspell.net/test/orig/batch0.tab. -Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) - -=============================================================================== - -The content of package org.apache.commons.codec.language.bm has been translated -from the original php source code available at http://stevemorse.org/phoneticinfo.htm -with permission from the original authors. -Original source copyright: -Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/modules/ingest-attachment/licenses/slf4j-api-LICENSE.txt b/modules/ingest-attachment/licenses/slf4j-api-LICENSE.txt deleted file mode 100644 index 52055e61de46f..0000000000000 --- a/modules/ingest-attachment/licenses/slf4j-api-LICENSE.txt +++ /dev/null @@ -1,21 +0,0 @@ -Copyright (c) 2004-2014 QOS.ch -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. diff --git a/modules/ingest-attachment/licenses/slf4j-api-NOTICE.txt b/modules/ingest-attachment/licenses/slf4j-api-NOTICE.txt deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/modules/ingest-geoip/licenses/jackson-LICENSE.txt b/modules/ingest-geoip/licenses/jackson-LICENSE.txt deleted file mode 100644 index f5f45d26a49d6..0000000000000 --- a/modules/ingest-geoip/licenses/jackson-LICENSE.txt +++ /dev/null @@ -1,8 +0,0 @@ -This copy of Jackson JSON processor streaming parser/generator is licensed under the -Apache (Software) License, version 2.0 ("the License"). -See the License for details about distribution rights, and the -specific rights regarding derivate works. - -You may obtain a copy of the License at: - -http://www.apache.org/licenses/LICENSE-2.0 diff --git a/modules/ingest-geoip/licenses/jackson-NOTICE.txt b/modules/ingest-geoip/licenses/jackson-NOTICE.txt deleted file mode 100644 index 4c976b7b4cc58..0000000000000 --- a/modules/ingest-geoip/licenses/jackson-NOTICE.txt +++ /dev/null @@ -1,20 +0,0 @@ -# Jackson JSON processor - -Jackson is a high-performance, Free/Open Source JSON processing library. -It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has -been in development since 2007. -It is currently developed by a community of developers, as well as supported -commercially by FasterXML.com. - -## Licensing - -Jackson core and extension components may licensed under different licenses. -To find the details that apply to this artifact see the accompanying LICENSE file. -For more information, including possible other licensing options, contact -FasterXML.com (http://fasterxml.com). - -## Credits - -A list of contributors may be found from CREDITS file, which is included -in some artifacts (usually source distributions); but is always available -from the source code management (SCM) system project uses. diff --git a/modules/legacy-geo/licenses/jackson-LICENSE.txt b/modules/legacy-geo/licenses/jackson-LICENSE.txt deleted file mode 100644 index f5f45d26a49d6..0000000000000 --- a/modules/legacy-geo/licenses/jackson-LICENSE.txt +++ /dev/null @@ -1,8 +0,0 @@ -This copy of Jackson JSON processor streaming parser/generator is licensed under the -Apache (Software) License, version 2.0 ("the License"). -See the License for details about distribution rights, and the -specific rights regarding derivate works. - -You may obtain a copy of the License at: - -http://www.apache.org/licenses/LICENSE-2.0 diff --git a/modules/legacy-geo/licenses/jackson-NOTICE.txt b/modules/legacy-geo/licenses/jackson-NOTICE.txt deleted file mode 100644 index 4c976b7b4cc58..0000000000000 --- a/modules/legacy-geo/licenses/jackson-NOTICE.txt +++ /dev/null @@ -1,20 +0,0 @@ -# Jackson JSON processor - -Jackson is a high-performance, Free/Open Source JSON processing library. -It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has -been in development since 2007. -It is currently developed by a community of developers, as well as supported -commercially by FasterXML.com. - -## Licensing - -Jackson core and extension components may licensed under different licenses. -To find the details that apply to this artifact see the accompanying LICENSE file. -For more information, including possible other licensing options, contact -FasterXML.com (http://fasterxml.com). - -## Credits - -A list of contributors may be found from CREDITS file, which is included -in some artifacts (usually source distributions); but is always available -from the source code management (SCM) system project uses. diff --git a/modules/repository-azure/licenses/slf4j-api-LICENSE.txt b/modules/repository-azure/licenses/slf4j-api-LICENSE.txt deleted file mode 100644 index 52055e61de46f..0000000000000 --- a/modules/repository-azure/licenses/slf4j-api-LICENSE.txt +++ /dev/null @@ -1,21 +0,0 @@ -Copyright (c) 2004-2014 QOS.ch -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. diff --git a/modules/repository-azure/licenses/slf4j-api-NOTICE.txt b/modules/repository-azure/licenses/slf4j-api-NOTICE.txt deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/modules/repository-gcs/licenses/commons-codec-LICENSE.txt b/modules/repository-gcs/licenses/commons-codec-LICENSE.txt deleted file mode 100644 index d645695673349..0000000000000 --- a/modules/repository-gcs/licenses/commons-codec-LICENSE.txt +++ /dev/null @@ -1,202 +0,0 @@ - - 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. diff --git a/modules/repository-gcs/licenses/commons-codec-NOTICE.txt b/modules/repository-gcs/licenses/commons-codec-NOTICE.txt deleted file mode 100644 index 56916449bbe10..0000000000000 --- a/modules/repository-gcs/licenses/commons-codec-NOTICE.txt +++ /dev/null @@ -1,17 +0,0 @@ -Apache Commons Codec -Copyright 2002-2015 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java -contains test data from http://aspell.net/test/orig/batch0.tab. -Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) - -=============================================================================== - -The content of package org.apache.commons.codec.language.bm has been translated -from the original php source code available at http://stevemorse.org/phoneticinfo.htm -with permission from the original authors. -Original source copyright: -Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/modules/repository-gcs/licenses/jackson-LICENSE.txt b/modules/repository-gcs/licenses/jackson-LICENSE.txt deleted file mode 100644 index f5f45d26a49d6..0000000000000 --- a/modules/repository-gcs/licenses/jackson-LICENSE.txt +++ /dev/null @@ -1,8 +0,0 @@ -This copy of Jackson JSON processor streaming parser/generator is licensed under the -Apache (Software) License, version 2.0 ("the License"). -See the License for details about distribution rights, and the -specific rights regarding derivate works. - -You may obtain a copy of the License at: - -http://www.apache.org/licenses/LICENSE-2.0 diff --git a/modules/repository-gcs/licenses/jackson-NOTICE.txt b/modules/repository-gcs/licenses/jackson-NOTICE.txt deleted file mode 100644 index 4c976b7b4cc58..0000000000000 --- a/modules/repository-gcs/licenses/jackson-NOTICE.txt +++ /dev/null @@ -1,20 +0,0 @@ -# Jackson JSON processor - -Jackson is a high-performance, Free/Open Source JSON processing library. -It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has -been in development since 2007. -It is currently developed by a community of developers, as well as supported -commercially by FasterXML.com. - -## Licensing - -Jackson core and extension components may licensed under different licenses. -To find the details that apply to this artifact see the accompanying LICENSE file. -For more information, including possible other licensing options, contact -FasterXML.com (http://fasterxml.com). - -## Credits - -A list of contributors may be found from CREDITS file, which is included -in some artifacts (usually source distributions); but is always available -from the source code management (SCM) system project uses. diff --git a/modules/repository-s3/licenses/commons-codec-LICENSE.txt b/modules/repository-s3/licenses/commons-codec-LICENSE.txt deleted file mode 100644 index d645695673349..0000000000000 --- a/modules/repository-s3/licenses/commons-codec-LICENSE.txt +++ /dev/null @@ -1,202 +0,0 @@ - - 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. diff --git a/modules/repository-s3/licenses/commons-codec-NOTICE.txt b/modules/repository-s3/licenses/commons-codec-NOTICE.txt deleted file mode 100644 index 56916449bbe10..0000000000000 --- a/modules/repository-s3/licenses/commons-codec-NOTICE.txt +++ /dev/null @@ -1,17 +0,0 @@ -Apache Commons Codec -Copyright 2002-2015 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java -contains test data from http://aspell.net/test/orig/batch0.tab. -Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) - -=============================================================================== - -The content of package org.apache.commons.codec.language.bm has been translated -from the original php source code available at http://stevemorse.org/phoneticinfo.htm -with permission from the original authors. -Original source copyright: -Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/modules/repository-s3/licenses/jackson-LICENSE b/modules/repository-s3/licenses/jackson-LICENSE deleted file mode 100644 index f5f45d26a49d6..0000000000000 --- a/modules/repository-s3/licenses/jackson-LICENSE +++ /dev/null @@ -1,8 +0,0 @@ -This copy of Jackson JSON processor streaming parser/generator is licensed under the -Apache (Software) License, version 2.0 ("the License"). -See the License for details about distribution rights, and the -specific rights regarding derivate works. - -You may obtain a copy of the License at: - -http://www.apache.org/licenses/LICENSE-2.0 diff --git a/modules/repository-s3/licenses/jackson-NOTICE b/modules/repository-s3/licenses/jackson-NOTICE deleted file mode 100644 index 4c976b7b4cc58..0000000000000 --- a/modules/repository-s3/licenses/jackson-NOTICE +++ /dev/null @@ -1,20 +0,0 @@ -# Jackson JSON processor - -Jackson is a high-performance, Free/Open Source JSON processing library. -It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has -been in development since 2007. -It is currently developed by a community of developers, as well as supported -commercially by FasterXML.com. - -## Licensing - -Jackson core and extension components may licensed under different licenses. -To find the details that apply to this artifact see the accompanying LICENSE file. -For more information, including possible other licensing options, contact -FasterXML.com (http://fasterxml.com). - -## Credits - -A list of contributors may be found from CREDITS file, which is included -in some artifacts (usually source distributions); but is always available -from the source code management (SCM) system project uses. diff --git a/modules/repository-url/licenses/commons-codec-LICENSE.txt b/modules/repository-url/licenses/commons-codec-LICENSE.txt deleted file mode 100644 index d645695673349..0000000000000 --- a/modules/repository-url/licenses/commons-codec-LICENSE.txt +++ /dev/null @@ -1,202 +0,0 @@ - - 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. diff --git a/modules/repository-url/licenses/commons-codec-NOTICE.txt b/modules/repository-url/licenses/commons-codec-NOTICE.txt deleted file mode 100644 index 56916449bbe10..0000000000000 --- a/modules/repository-url/licenses/commons-codec-NOTICE.txt +++ /dev/null @@ -1,17 +0,0 @@ -Apache Commons Codec -Copyright 2002-2015 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java -contains test data from http://aspell.net/test/orig/batch0.tab. -Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) - -=============================================================================== - -The content of package org.apache.commons.codec.language.bm has been translated -from the original php source code available at http://stevemorse.org/phoneticinfo.htm -with permission from the original authors. -Original source copyright: -Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/plugins/analysis-phonetic/licenses/commons-codec-LICENSE.txt b/plugins/analysis-phonetic/licenses/commons-codec-LICENSE.txt deleted file mode 100644 index 57bc88a15a0ee..0000000000000 --- a/plugins/analysis-phonetic/licenses/commons-codec-LICENSE.txt +++ /dev/null @@ -1,202 +0,0 @@ - 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. - diff --git a/plugins/analysis-phonetic/licenses/commons-codec-NOTICE.txt b/plugins/analysis-phonetic/licenses/commons-codec-NOTICE.txt deleted file mode 100644 index 72eb32a902458..0000000000000 --- a/plugins/analysis-phonetic/licenses/commons-codec-NOTICE.txt +++ /dev/null @@ -1,5 +0,0 @@ -Apache Commons CLI -Copyright 2001-2009 The Apache Software Foundation - -This product includes software developed by -The Apache Software Foundation (http://www.apache.org/). diff --git a/plugins/discovery-azure-classic/licenses/commons-codec-LICENSE.txt b/plugins/discovery-azure-classic/licenses/commons-codec-LICENSE.txt deleted file mode 100644 index d645695673349..0000000000000 --- a/plugins/discovery-azure-classic/licenses/commons-codec-LICENSE.txt +++ /dev/null @@ -1,202 +0,0 @@ - - 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. diff --git a/plugins/discovery-azure-classic/licenses/commons-codec-NOTICE.txt b/plugins/discovery-azure-classic/licenses/commons-codec-NOTICE.txt deleted file mode 100644 index 56916449bbe10..0000000000000 --- a/plugins/discovery-azure-classic/licenses/commons-codec-NOTICE.txt +++ /dev/null @@ -1,17 +0,0 @@ -Apache Commons Codec -Copyright 2002-2015 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java -contains test data from http://aspell.net/test/orig/batch0.tab. -Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) - -=============================================================================== - -The content of package org.apache.commons.codec.language.bm has been translated -from the original php source code available at http://stevemorse.org/phoneticinfo.htm -with permission from the original authors. -Original source copyright: -Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/plugins/discovery-ec2/licenses/commons-codec-LICENSE.txt b/plugins/discovery-ec2/licenses/commons-codec-LICENSE.txt deleted file mode 100644 index d645695673349..0000000000000 --- a/plugins/discovery-ec2/licenses/commons-codec-LICENSE.txt +++ /dev/null @@ -1,202 +0,0 @@ - - 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. diff --git a/plugins/discovery-ec2/licenses/commons-codec-NOTICE.txt b/plugins/discovery-ec2/licenses/commons-codec-NOTICE.txt deleted file mode 100644 index 56916449bbe10..0000000000000 --- a/plugins/discovery-ec2/licenses/commons-codec-NOTICE.txt +++ /dev/null @@ -1,17 +0,0 @@ -Apache Commons Codec -Copyright 2002-2015 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java -contains test data from http://aspell.net/test/orig/batch0.tab. -Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) - -=============================================================================== - -The content of package org.apache.commons.codec.language.bm has been translated -from the original php source code available at http://stevemorse.org/phoneticinfo.htm -with permission from the original authors. -Original source copyright: -Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/plugins/discovery-ec2/licenses/jackson-LICENSE b/plugins/discovery-ec2/licenses/jackson-LICENSE deleted file mode 100644 index f5f45d26a49d6..0000000000000 --- a/plugins/discovery-ec2/licenses/jackson-LICENSE +++ /dev/null @@ -1,8 +0,0 @@ -This copy of Jackson JSON processor streaming parser/generator is licensed under the -Apache (Software) License, version 2.0 ("the License"). -See the License for details about distribution rights, and the -specific rights regarding derivate works. - -You may obtain a copy of the License at: - -http://www.apache.org/licenses/LICENSE-2.0 diff --git a/plugins/discovery-ec2/licenses/jackson-NOTICE b/plugins/discovery-ec2/licenses/jackson-NOTICE deleted file mode 100644 index 4c976b7b4cc58..0000000000000 --- a/plugins/discovery-ec2/licenses/jackson-NOTICE +++ /dev/null @@ -1,20 +0,0 @@ -# Jackson JSON processor - -Jackson is a high-performance, Free/Open Source JSON processing library. -It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has -been in development since 2007. -It is currently developed by a community of developers, as well as supported -commercially by FasterXML.com. - -## Licensing - -Jackson core and extension components may licensed under different licenses. -To find the details that apply to this artifact see the accompanying LICENSE file. -For more information, including possible other licensing options, contact -FasterXML.com (http://fasterxml.com). - -## Credits - -A list of contributors may be found from CREDITS file, which is included -in some artifacts (usually source distributions); but is always available -from the source code management (SCM) system project uses. diff --git a/plugins/discovery-gce/licenses/commons-codec-LICENSE.txt b/plugins/discovery-gce/licenses/commons-codec-LICENSE.txt deleted file mode 100644 index d645695673349..0000000000000 --- a/plugins/discovery-gce/licenses/commons-codec-LICENSE.txt +++ /dev/null @@ -1,202 +0,0 @@ - - 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. diff --git a/plugins/discovery-gce/licenses/commons-codec-NOTICE.txt b/plugins/discovery-gce/licenses/commons-codec-NOTICE.txt deleted file mode 100644 index 56916449bbe10..0000000000000 --- a/plugins/discovery-gce/licenses/commons-codec-NOTICE.txt +++ /dev/null @@ -1,17 +0,0 @@ -Apache Commons Codec -Copyright 2002-2015 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java -contains test data from http://aspell.net/test/orig/batch0.tab. -Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) - -=============================================================================== - -The content of package org.apache.commons.codec.language.bm has been translated -from the original php source code available at http://stevemorse.org/phoneticinfo.htm -with permission from the original authors. -Original source copyright: -Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/plugins/discovery-gce/licenses/jackson-LICENSE.txt b/plugins/discovery-gce/licenses/jackson-LICENSE.txt deleted file mode 100644 index f5f45d26a49d6..0000000000000 --- a/plugins/discovery-gce/licenses/jackson-LICENSE.txt +++ /dev/null @@ -1,8 +0,0 @@ -This copy of Jackson JSON processor streaming parser/generator is licensed under the -Apache (Software) License, version 2.0 ("the License"). -See the License for details about distribution rights, and the -specific rights regarding derivate works. - -You may obtain a copy of the License at: - -http://www.apache.org/licenses/LICENSE-2.0 diff --git a/plugins/discovery-gce/licenses/jackson-NOTICE.txt b/plugins/discovery-gce/licenses/jackson-NOTICE.txt deleted file mode 100644 index 4c976b7b4cc58..0000000000000 --- a/plugins/discovery-gce/licenses/jackson-NOTICE.txt +++ /dev/null @@ -1,20 +0,0 @@ -# Jackson JSON processor - -Jackson is a high-performance, Free/Open Source JSON processing library. -It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has -been in development since 2007. -It is currently developed by a community of developers, as well as supported -commercially by FasterXML.com. - -## Licensing - -Jackson core and extension components may licensed under different licenses. -To find the details that apply to this artifact see the accompanying LICENSE file. -For more information, including possible other licensing options, contact -FasterXML.com (http://fasterxml.com). - -## Credits - -A list of contributors may be found from CREDITS file, which is included -in some artifacts (usually source distributions); but is always available -from the source code management (SCM) system project uses. diff --git a/plugins/repository-hdfs/licenses/commons-codec-LICENSE.txt b/plugins/repository-hdfs/licenses/commons-codec-LICENSE.txt deleted file mode 100644 index 75b52484ea471..0000000000000 --- a/plugins/repository-hdfs/licenses/commons-codec-LICENSE.txt +++ /dev/null @@ -1,202 +0,0 @@ - - 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. diff --git a/plugins/repository-hdfs/licenses/commons-codec-NOTICE.txt b/plugins/repository-hdfs/licenses/commons-codec-NOTICE.txt deleted file mode 100644 index efc098ca3ee67..0000000000000 --- a/plugins/repository-hdfs/licenses/commons-codec-NOTICE.txt +++ /dev/null @@ -1,17 +0,0 @@ -Apache Commons Codec -Copyright 2002-2014 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java -contains test data from http://aspell.net/test/orig/batch0.tab. -Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) - -=============================================================================== - -The content of package org.apache.commons.codec.language.bm has been translated -from the original php source code available at http://stevemorse.org/phoneticinfo.htm -with permission from the original authors. -Original source copyright: -Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/plugins/repository-hdfs/licenses/slf4j-api-LICENSE.txt b/plugins/repository-hdfs/licenses/slf4j-api-LICENSE.txt deleted file mode 100644 index 8fda22f4d72f6..0000000000000 --- a/plugins/repository-hdfs/licenses/slf4j-api-LICENSE.txt +++ /dev/null @@ -1,21 +0,0 @@ -Copyright (c) 2004-2014 QOS.ch -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. diff --git a/plugins/repository-hdfs/licenses/slf4j-api-NOTICE.txt b/plugins/repository-hdfs/licenses/slf4j-api-NOTICE.txt deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/x-pack/plugin/ent-search/licenses/jackson-LICENSE b/x-pack/plugin/ent-search/licenses/jackson-LICENSE deleted file mode 100644 index f5f45d26a49d6..0000000000000 --- a/x-pack/plugin/ent-search/licenses/jackson-LICENSE +++ /dev/null @@ -1,8 +0,0 @@ -This copy of Jackson JSON processor streaming parser/generator is licensed under the -Apache (Software) License, version 2.0 ("the License"). -See the License for details about distribution rights, and the -specific rights regarding derivate works. - -You may obtain a copy of the License at: - -http://www.apache.org/licenses/LICENSE-2.0 diff --git a/x-pack/plugin/ent-search/licenses/jackson-NOTICE b/x-pack/plugin/ent-search/licenses/jackson-NOTICE deleted file mode 100644 index 4c976b7b4cc58..0000000000000 --- a/x-pack/plugin/ent-search/licenses/jackson-NOTICE +++ /dev/null @@ -1,20 +0,0 @@ -# Jackson JSON processor - -Jackson is a high-performance, Free/Open Source JSON processing library. -It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has -been in development since 2007. -It is currently developed by a community of developers, as well as supported -commercially by FasterXML.com. - -## Licensing - -Jackson core and extension components may licensed under different licenses. -To find the details that apply to this artifact see the accompanying LICENSE file. -For more information, including possible other licensing options, contact -FasterXML.com (http://fasterxml.com). - -## Credits - -A list of contributors may be found from CREDITS file, which is included -in some artifacts (usually source distributions); but is always available -from the source code management (SCM) system project uses. diff --git a/x-pack/plugin/ent-search/licenses/slf4j-api-LICENSE.txt b/x-pack/plugin/ent-search/licenses/slf4j-api-LICENSE.txt deleted file mode 100644 index 52055e61de46f..0000000000000 --- a/x-pack/plugin/ent-search/licenses/slf4j-api-LICENSE.txt +++ /dev/null @@ -1,21 +0,0 @@ -Copyright (c) 2004-2014 QOS.ch -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. diff --git a/x-pack/plugin/ent-search/licenses/slf4j-api-NOTICE.txt b/x-pack/plugin/ent-search/licenses/slf4j-api-NOTICE.txt deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/x-pack/plugin/identity-provider/licenses/slf4j-api-LICENSE.txt b/x-pack/plugin/identity-provider/licenses/slf4j-api-LICENSE.txt deleted file mode 100644 index c8e72c2cd4738..0000000000000 --- a/x-pack/plugin/identity-provider/licenses/slf4j-api-LICENSE.txt +++ /dev/null @@ -1,21 +0,0 @@ - Copyright (c) 2004-2017 QOS.ch - 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. diff --git a/x-pack/plugin/identity-provider/licenses/slf4j-api-NOTICE.txt b/x-pack/plugin/identity-provider/licenses/slf4j-api-NOTICE.txt deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/x-pack/plugin/inference/licenses/jackson-LICENSE.txt b/x-pack/plugin/inference/licenses/jackson-LICENSE.txt deleted file mode 100644 index f5f45d26a49d6..0000000000000 --- a/x-pack/plugin/inference/licenses/jackson-LICENSE.txt +++ /dev/null @@ -1,8 +0,0 @@ -This copy of Jackson JSON processor streaming parser/generator is licensed under the -Apache (Software) License, version 2.0 ("the License"). -See the License for details about distribution rights, and the -specific rights regarding derivate works. - -You may obtain a copy of the License at: - -http://www.apache.org/licenses/LICENSE-2.0 diff --git a/x-pack/plugin/inference/licenses/jackson-NOTICE.txt b/x-pack/plugin/inference/licenses/jackson-NOTICE.txt deleted file mode 100644 index 4c976b7b4cc58..0000000000000 --- a/x-pack/plugin/inference/licenses/jackson-NOTICE.txt +++ /dev/null @@ -1,20 +0,0 @@ -# Jackson JSON processor - -Jackson is a high-performance, Free/Open Source JSON processing library. -It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has -been in development since 2007. -It is currently developed by a community of developers, as well as supported -commercially by FasterXML.com. - -## Licensing - -Jackson core and extension components may licensed under different licenses. -To find the details that apply to this artifact see the accompanying LICENSE file. -For more information, including possible other licensing options, contact -FasterXML.com (http://fasterxml.com). - -## Credits - -A list of contributors may be found from CREDITS file, which is included -in some artifacts (usually source distributions); but is always available -from the source code management (SCM) system project uses. diff --git a/x-pack/plugin/security/licenses/slf4j-api-LICENSE.txt b/x-pack/plugin/security/licenses/slf4j-api-LICENSE.txt deleted file mode 100644 index c8e72c2cd4738..0000000000000 --- a/x-pack/plugin/security/licenses/slf4j-api-LICENSE.txt +++ /dev/null @@ -1,21 +0,0 @@ - Copyright (c) 2004-2017 QOS.ch - 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. diff --git a/x-pack/plugin/security/licenses/slf4j-api-NOTICE.txt b/x-pack/plugin/security/licenses/slf4j-api-NOTICE.txt deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/x-pack/plugin/vector-tile/licenses/slf4j-api-LICENSE.txt b/x-pack/plugin/vector-tile/licenses/slf4j-api-LICENSE.txt deleted file mode 100644 index c8e72c2cd4738..0000000000000 --- a/x-pack/plugin/vector-tile/licenses/slf4j-api-LICENSE.txt +++ /dev/null @@ -1,21 +0,0 @@ - Copyright (c) 2004-2017 QOS.ch - 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. diff --git a/x-pack/plugin/vector-tile/licenses/slf4j-api-NOTICE.txt b/x-pack/plugin/vector-tile/licenses/slf4j-api-NOTICE.txt deleted file mode 100644 index e69de29bb2d1d..0000000000000 From af76ead787b992b1525367b78fc9f53812dcac52 Mon Sep 17 00:00:00 2001 From: Sylvain Wallez Date: Wed, 19 Mar 2025 14:56:48 +0100 Subject: [PATCH 07/19] Fix server entitlements --- .../initialization/EntitlementInitialization.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java index 94a8629f7f619..a75c1d169a8f4 100644 --- a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java +++ b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java @@ -270,13 +270,15 @@ private static PolicyManager createPolicyManager() { ); } - // In s3-repository, aws-java-sdk-core reads its default configuration from a json file in its jar. - // Its URL is opened by Jackson, causing an entitlement failure in `(server)`. + // In modules:repository-s3 and plugins:discovery-ec2, aws-java-sdk-core reads its default configuration from a json file in + // its jar. Its URL is opened by Jackson, causing an entitlement failure in `(server)`. Collections.addAll( serverScopes, new Scope( - "com.fasterxml.jackson.core", - List.of(new FilesEntitlement(List.of(FileData.ofPath(bootstrapArgs.modulesDir(), READ)))) + "com.fasterxml.jackson.core", List.of( + new FilesEntitlement(List.of(FileData.ofPath(bootstrapArgs.modulesDir(), READ))), + new FilesEntitlement(List.of(FileData.ofPath(bootstrapArgs.pluginsDir(), READ))) + ) ) ); From 5510ae840b9d68ca9fc397053219957f188e0fb7 Mon Sep 17 00:00:00 2001 From: Sylvain Wallez Date: Wed, 19 Mar 2025 15:39:18 +0100 Subject: [PATCH 08/19] Fix forbiddanApis task --- libs/arrow/build.gradle | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/libs/arrow/build.gradle b/libs/arrow/build.gradle index 156fca8dd84b1..b2a9ff56d3c33 100644 --- a/libs/arrow/build.gradle +++ b/libs/arrow/build.gradle @@ -11,6 +11,8 @@ // - additional JVM arguments are added to distribution/src/config/jvm.options and ElasticsearchTestBasePlugin // - additional permissions are added to server/src/main/resources/org/elasticsearch/bootstrap/security.policy +import org.elasticsearch.gradle.internal.precommit.CheckForbiddenApisTask + apply plugin: 'elasticsearch.build' apply plugin: 'elasticsearch.publish' @@ -82,3 +84,9 @@ tasks.named("thirdPartyAudit").configure { 'org.apache.arrow.memory.util.ByteFunctionHelpers', ) } + +tasks.withType(CheckForbiddenApisTask).configureEach { + // Remove server signatures as they will fail on classes missing in this lib's classpath, + // like org.apache.lucene.util.IOUtils + replaceSignatureFiles('jdk-signatures') +} From 57186c839464f0d3616ba1ea5e29e013f6f9eae4 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Wed, 19 Mar 2025 14:47:58 +0000 Subject: [PATCH 09/19] [CI] Auto commit changes from spotless --- .../entitlement/initialization/EntitlementInitialization.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java index a75c1d169a8f4..15151ebc135bf 100644 --- a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java +++ b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java @@ -275,7 +275,8 @@ private static PolicyManager createPolicyManager() { Collections.addAll( serverScopes, new Scope( - "com.fasterxml.jackson.core", List.of( + "com.fasterxml.jackson.core", + List.of( new FilesEntitlement(List.of(FileData.ofPath(bootstrapArgs.modulesDir(), READ))), new FilesEntitlement(List.of(FileData.ofPath(bootstrapArgs.pluginsDir(), READ))) ) From 5952435bb5a2570b61b0ecb032e7e9f9ebbb7cdd Mon Sep 17 00:00:00 2001 From: Sylvain Wallez Date: Wed, 19 Mar 2025 16:36:43 +0100 Subject: [PATCH 10/19] Fix server entitlements (hopefully for good) --- .../initialization/EntitlementInitialization.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java index 15151ebc135bf..a2084fa556a47 100644 --- a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java +++ b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java @@ -275,11 +275,10 @@ private static PolicyManager createPolicyManager() { Collections.addAll( serverScopes, new Scope( - "com.fasterxml.jackson.core", - List.of( - new FilesEntitlement(List.of(FileData.ofPath(bootstrapArgs.modulesDir(), READ))), - new FilesEntitlement(List.of(FileData.ofPath(bootstrapArgs.pluginsDir(), READ))) - ) + "com.fasterxml.jackson.core", List.of(new FilesEntitlement(List.of( + FileData.ofPath(bootstrapArgs.modulesDir(), READ), + FileData.ofPath(bootstrapArgs.pluginsDir(), READ) + ))) ) ); From 259ed8a63914a5d9178de9c188bd2cc93a61baad Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Wed, 19 Mar 2025 15:46:42 +0000 Subject: [PATCH 11/19] [CI] Auto commit changes from spotless --- .../initialization/EntitlementInitialization.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java index a2084fa556a47..d25f76423bceb 100644 --- a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java +++ b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java @@ -275,10 +275,12 @@ private static PolicyManager createPolicyManager() { Collections.addAll( serverScopes, new Scope( - "com.fasterxml.jackson.core", List.of(new FilesEntitlement(List.of( - FileData.ofPath(bootstrapArgs.modulesDir(), READ), - FileData.ofPath(bootstrapArgs.pluginsDir(), READ) - ))) + "com.fasterxml.jackson.core", + List.of( + new FilesEntitlement( + List.of(FileData.ofPath(bootstrapArgs.modulesDir(), READ), FileData.ofPath(bootstrapArgs.pluginsDir(), READ)) + ) + ) ) ); From 40c317625fc2defeae9a4ced37dc484d513b2330 Mon Sep 17 00:00:00 2001 From: Sylvain Wallez Date: Wed, 19 Mar 2025 17:11:09 +0100 Subject: [PATCH 12/19] Fix thirdPartyAudit --- libs/arrow/build.gradle | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/libs/arrow/build.gradle b/libs/arrow/build.gradle index b2a9ff56d3c33..5d4e5f430c455 100644 --- a/libs/arrow/build.gradle +++ b/libs/arrow/build.gradle @@ -70,18 +70,8 @@ tasks.named("dependencyLicenses").configure { tasks.named("thirdPartyAudit").configure { ignoreViolations( - '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', ) } From 21cb8ff657c88020fe998196eeb5725bd1a909cf Mon Sep 17 00:00:00 2001 From: Sylvain Wallez Date: Wed, 19 Mar 2025 19:22:45 +0100 Subject: [PATCH 13/19] Refactor: move Arrow bulk out of the server module. This causes too many version conflicts. --- .idea/inspectionProfiles/Project_Default.xml | 5 +- build-tools-internal/version.properties | 4 +- distribution/src/config/jvm.options | 4 +- gradle/build.versions.toml | 2 +- gradle/verification-metadata.xml | 35 -- libs/arrow/build.gradle | 10 +- libs/arrow/licenses/commons-codec-NOTICE.txt | 2 +- .../src/main/java/module-info.java-disabled | 35 ++ .../elasticsearch/{ => libs}/arrow/Arrow.java | 22 +- .../arrow/ArrowFormatException.java | 2 +- .../EntitlementInitialization.java | 14 - libs/x-content/impl/build.gradle | 10 +- .../impl/src/main/java/module-info.java | 2 +- modules/arrow/build.gradle | 34 ++ modules/arrow/licenses/jackson-LICENSE.txt | 202 ++++++++ modules/arrow/licenses/jackson-NOTICE.txt | 20 + .../src/main/java/module-info.java-disabled | 16 +- .../org/elasticsearch/arrow/ArrowPlugin.java | 74 +++ .../arrow/bulk/ArrowBulkAction.java | 66 +++ .../bulk}/ArrowBulkIncrementalParser.java | 12 +- .../arrow/bulk}/ArrowBulkRequestParser.java | 13 +- .../arrow/bulk}/ArrowIncrementalParser.java | 8 +- .../arrow/bulk}/BytesReferenceChannel.java | 2 +- .../arrow/xcontent}/ArrowJsonParser.java | 3 +- .../xcontent}/ArrowJsonXContentParser.java | 2 +- .../arrow/xcontent}/ArrowToXContent.java | 3 +- .../arrow/xcontent}/XContentBuffer.java | 2 +- .../arrow/xcontent}/ArrowToXContentTests.java | 7 +- modules/ingest-attachment/build.gradle | 80 +-- .../licenses/commons-codec-LICENSE.txt | 202 ++++++++ .../licenses/commons-codec-NOTICE.txt | 17 + .../licenses/slf4j-api-LICENSE.txt | 21 + .../licenses/slf4j-api-NOTICE.txt | 0 .../ingest-geoip/licenses/jackson-LICENSE.txt | 8 + .../ingest-geoip/licenses/jackson-NOTICE.txt | 20 + .../legacy-geo/licenses/jackson-LICENSE.txt | 8 + .../legacy-geo/licenses/jackson-NOTICE.txt | 20 + .../licenses/slf4j-api-LICENSE.txt | 21 + .../licenses/slf4j-api-NOTICE.txt | 0 .../licenses/commons-codec-LICENSE.txt | 202 ++++++++ .../licenses/commons-codec-NOTICE.txt | 17 + .../licenses/jackson-LICENSE.txt | 8 + .../licenses/jackson-NOTICE.txt | 20 + .../licenses/commons-codec-LICENSE.txt | 202 ++++++++ .../licenses/commons-codec-NOTICE.txt | 17 + .../repository-s3/licenses/jackson-LICENSE | 8 + modules/repository-s3/licenses/jackson-NOTICE | 20 + .../licenses/commons-codec-LICENSE.txt | 202 ++++++++ .../licenses/commons-codec-NOTICE.txt | 17 + .../licenses/commons-codec-LICENSE.txt | 202 ++++++++ .../licenses/commons-codec-NOTICE.txt | 5 + .../licenses/commons-codec-LICENSE.txt | 202 ++++++++ .../licenses/commons-codec-NOTICE.txt | 17 + .../licenses/commons-codec-LICENSE.txt | 202 ++++++++ .../licenses/commons-codec-NOTICE.txt | 17 + .../discovery-ec2/licenses/jackson-LICENSE | 8 + plugins/discovery-ec2/licenses/jackson-NOTICE | 20 + .../licenses/commons-codec-LICENSE.txt | 202 ++++++++ .../licenses/commons-codec-NOTICE.txt | 17 + .../licenses/jackson-LICENSE.txt | 8 + .../discovery-gce/licenses/jackson-NOTICE.txt | 20 + .../licenses/commons-codec-LICENSE.txt | 202 ++++++++ .../licenses/commons-codec-NOTICE.txt | 17 + .../licenses/slf4j-api-LICENSE.txt | 21 + .../licenses/slf4j-api-NOTICE.txt | 0 .../rest-api-spec/api/arrow.bulk.json | 98 ++++ server/build.gradle | 3 - .../action/bulk/arrow/BulkArrowIT.java | 182 ------- server/src/main/java/module-info.java | 8 +- .../action/bulk/BulkRequestParser.java | 34 +- .../elasticsearch/rest/RestController.java | 5 +- .../rest/action/document/RestBulkAction.java | 23 +- .../elasticsearch/bootstrap/security.policy | 5 - .../ArrowBulkIncrementalParserTests.java | 473 ------------------ .../arrow/BytesReferenceChannelTests.java | 50 -- .../ent-search/licenses/jackson-LICENSE | 8 + .../plugin/ent-search/licenses/jackson-NOTICE | 20 + .../ent-search/licenses/slf4j-api-LICENSE.txt | 21 + .../ent-search/licenses/slf4j-api-NOTICE.txt | 0 x-pack/plugin/esql/arrow/build.gradle | 6 +- .../esql/qa/server/single-node/build.gradle | 13 +- .../licenses/slf4j-api-LICENSE.txt | 21 + .../licenses/slf4j-api-NOTICE.txt | 0 .../inference/licenses/jackson-LICENSE.txt | 8 + .../inference/licenses/jackson-NOTICE.txt | 20 + .../security/licenses/slf4j-api-LICENSE.txt | 21 + .../security/licenses/slf4j-api-NOTICE.txt | 0 .../licenses/slf4j-api-LICENSE.txt | 21 + .../vector-tile/licenses/slf4j-api-NOTICE.txt | 0 89 files changed, 3011 insertions(+), 910 deletions(-) create mode 100644 libs/arrow/src/main/java/module-info.java-disabled rename libs/arrow/src/main/java/org/elasticsearch/{ => libs}/arrow/Arrow.java (53%) rename libs/arrow/src/main/java/org/elasticsearch/{ => libs}/arrow/ArrowFormatException.java (94%) create mode 100644 modules/arrow/build.gradle create mode 100644 modules/arrow/licenses/jackson-LICENSE.txt create mode 100644 modules/arrow/licenses/jackson-NOTICE.txt rename libs/arrow/src/main/java/module-info.java => modules/arrow/src/main/java/module-info.java-disabled (62%) create mode 100644 modules/arrow/src/main/java/org/elasticsearch/arrow/ArrowPlugin.java create mode 100644 modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkAction.java rename {server/src/main/java/org/elasticsearch/action/bulk/arrow => modules/arrow/src/main/java/org/elasticsearch/arrow/bulk}/ArrowBulkIncrementalParser.java (97%) rename {server/src/main/java/org/elasticsearch/action/bulk/arrow => modules/arrow/src/main/java/org/elasticsearch/arrow/bulk}/ArrowBulkRequestParser.java (91%) rename {server/src/main/java/org/elasticsearch/action/bulk/arrow => modules/arrow/src/main/java/org/elasticsearch/arrow/bulk}/ArrowIncrementalParser.java (95%) rename {server/src/main/java/org/elasticsearch/action/bulk/arrow => modules/arrow/src/main/java/org/elasticsearch/arrow/bulk}/BytesReferenceChannel.java (98%) rename {libs/arrow/src/main/java/org/elasticsearch/arrow => modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent}/ArrowJsonParser.java (98%) rename {libs/arrow/src/main/java/org/elasticsearch/arrow => modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent}/ArrowJsonXContentParser.java (99%) rename {libs/arrow/src/main/java/org/elasticsearch/arrow => modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent}/ArrowToXContent.java (99%) rename {libs/arrow/src/main/java/org/elasticsearch/arrow => modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent}/XContentBuffer.java (99%) rename {libs/arrow/src/test/java/org/elasticsearch/arrow => modules/arrow/src/test/java/org/elasticsearch/arrow/xcontent}/ArrowToXContentTests.java (93%) create mode 100644 modules/ingest-attachment/licenses/commons-codec-LICENSE.txt create mode 100644 modules/ingest-attachment/licenses/commons-codec-NOTICE.txt create mode 100644 modules/ingest-attachment/licenses/slf4j-api-LICENSE.txt create mode 100644 modules/ingest-attachment/licenses/slf4j-api-NOTICE.txt create mode 100644 modules/ingest-geoip/licenses/jackson-LICENSE.txt create mode 100644 modules/ingest-geoip/licenses/jackson-NOTICE.txt create mode 100644 modules/legacy-geo/licenses/jackson-LICENSE.txt create mode 100644 modules/legacy-geo/licenses/jackson-NOTICE.txt create mode 100644 modules/repository-azure/licenses/slf4j-api-LICENSE.txt create mode 100644 modules/repository-azure/licenses/slf4j-api-NOTICE.txt create mode 100644 modules/repository-gcs/licenses/commons-codec-LICENSE.txt create mode 100644 modules/repository-gcs/licenses/commons-codec-NOTICE.txt create mode 100644 modules/repository-gcs/licenses/jackson-LICENSE.txt create mode 100644 modules/repository-gcs/licenses/jackson-NOTICE.txt create mode 100644 modules/repository-s3/licenses/commons-codec-LICENSE.txt create mode 100644 modules/repository-s3/licenses/commons-codec-NOTICE.txt create mode 100644 modules/repository-s3/licenses/jackson-LICENSE create mode 100644 modules/repository-s3/licenses/jackson-NOTICE create mode 100644 modules/repository-url/licenses/commons-codec-LICENSE.txt create mode 100644 modules/repository-url/licenses/commons-codec-NOTICE.txt create mode 100644 plugins/analysis-phonetic/licenses/commons-codec-LICENSE.txt create mode 100644 plugins/analysis-phonetic/licenses/commons-codec-NOTICE.txt create mode 100644 plugins/discovery-azure-classic/licenses/commons-codec-LICENSE.txt create mode 100644 plugins/discovery-azure-classic/licenses/commons-codec-NOTICE.txt create mode 100644 plugins/discovery-ec2/licenses/commons-codec-LICENSE.txt create mode 100644 plugins/discovery-ec2/licenses/commons-codec-NOTICE.txt create mode 100644 plugins/discovery-ec2/licenses/jackson-LICENSE create mode 100644 plugins/discovery-ec2/licenses/jackson-NOTICE create mode 100644 plugins/discovery-gce/licenses/commons-codec-LICENSE.txt create mode 100644 plugins/discovery-gce/licenses/commons-codec-NOTICE.txt create mode 100644 plugins/discovery-gce/licenses/jackson-LICENSE.txt create mode 100644 plugins/discovery-gce/licenses/jackson-NOTICE.txt create mode 100644 plugins/repository-hdfs/licenses/commons-codec-LICENSE.txt create mode 100644 plugins/repository-hdfs/licenses/commons-codec-NOTICE.txt create mode 100644 plugins/repository-hdfs/licenses/slf4j-api-LICENSE.txt create mode 100644 plugins/repository-hdfs/licenses/slf4j-api-NOTICE.txt create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/api/arrow.bulk.json delete mode 100644 server/src/javaRestTest/java/org/elasticsearch/action/bulk/arrow/BulkArrowIT.java delete mode 100644 server/src/test/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParserTests.java delete mode 100644 server/src/test/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannelTests.java create mode 100644 x-pack/plugin/ent-search/licenses/jackson-LICENSE create mode 100644 x-pack/plugin/ent-search/licenses/jackson-NOTICE create mode 100644 x-pack/plugin/ent-search/licenses/slf4j-api-LICENSE.txt create mode 100644 x-pack/plugin/ent-search/licenses/slf4j-api-NOTICE.txt create mode 100644 x-pack/plugin/identity-provider/licenses/slf4j-api-LICENSE.txt create mode 100644 x-pack/plugin/identity-provider/licenses/slf4j-api-NOTICE.txt create mode 100644 x-pack/plugin/inference/licenses/jackson-LICENSE.txt create mode 100644 x-pack/plugin/inference/licenses/jackson-NOTICE.txt create mode 100644 x-pack/plugin/security/licenses/slf4j-api-LICENSE.txt create mode 100644 x-pack/plugin/security/licenses/slf4j-api-NOTICE.txt create mode 100644 x-pack/plugin/vector-tile/licenses/slf4j-api-LICENSE.txt create mode 100644 x-pack/plugin/vector-tile/licenses/slf4j-api-NOTICE.txt diff --git a/.idea/inspectionProfiles/Project_Default.xml b/.idea/inspectionProfiles/Project_Default.xml index 9e9a0f2dc634f..34ff98c18372f 100644 --- a/.idea/inspectionProfiles/Project_Default.xml +++ b/.idea/inspectionProfiles/Project_Default.xml @@ -1,9 +1,6 @@ - \ No newline at end of file + diff --git a/build-tools-internal/version.properties b/build-tools-internal/version.properties index bfee3fbea6782..a0341c9c5035f 100644 --- a/build-tools-internal/version.properties +++ b/build-tools-internal/version.properties @@ -11,7 +11,7 @@ snakeyaml = 2.0 icu4j = 68.2 supercsv = 2.4.0 log4j = 2.19.0 -slf4j = 2.0.10 +slf4j = 2.0.6 ecsLogging = 1.2.0 jna = 5.12.1 netty = 4.1.118.Final @@ -32,7 +32,7 @@ httpclient = 4.5.14 httpcore = 4.4.16 httpasyncclient = 4.1.5 commonslogging = 1.2 -commonscodec = 1.17.1 +commonscodec = 1.15 protobuf = 3.25.5 # test dependencies diff --git a/distribution/src/config/jvm.options b/distribution/src/config/jvm.options index 786c0a81c5540..1ed69a7d06c41 100644 --- a/distribution/src/config/jvm.options +++ b/distribution/src/config/jvm.options @@ -88,6 +88,8 @@ ## Arrow # Allow accessing a private field of java.nio.Buffer for direct memory access. # See org.apache.arrow.memory.MemoryUtil and https://arrow.apache.org/docs/java/install.html ---add-opens=java.base/java.nio=org.apache.arrow.memory.core +# See also libs/arrow/src/main/java/module-info.java-disabled for why we open to ALL-UNNAMED +# instead of limiting to org.apache.arrow.memory.core +--add-opens=java.base/java.nio=ALL-UNNAMED # Define the allocation manager type to avoid classpath scanning to locate one. -Darrow.allocation.manager.type=Unsafe diff --git a/gradle/build.versions.toml b/gradle/build.versions.toml index 2964235e79241..3635e26dcc21c 100644 --- a/gradle/build.versions.toml +++ b/gradle/build.versions.toml @@ -1,6 +1,6 @@ [versions] asm = "9.7.1" -jackson = "2.17.2" +jackson = "2.15.0" junit5 = "5.8.1" spock = "2.1-groovy-3.0" diff --git a/gradle/verification-metadata.xml b/gradle/verification-metadata.xml index f62b9ea3a0455..207275dfe3468 100644 --- a/gradle/verification-metadata.xml +++ b/gradle/verification-metadata.xml @@ -294,11 +294,6 @@ - - - - - @@ -349,11 +344,6 @@ - - - - - @@ -639,11 +629,6 @@ - - - - - @@ -1239,11 +1224,6 @@ - - - - - @@ -3921,11 +3901,6 @@ - - - - - @@ -4526,11 +4501,6 @@ - - - - - @@ -4591,11 +4561,6 @@ - - - - - diff --git a/libs/arrow/build.gradle b/libs/arrow/build.gradle index 5d4e5f430c455..e722668bad43f 100644 --- a/libs/arrow/build.gradle +++ b/libs/arrow/build.gradle @@ -19,7 +19,7 @@ apply plugin: 'elasticsearch.publish' var arrowVersion = "18.2.0" dependencies { - implementation(project(":libs:x-content")) + //implementation(project(":libs:x-content")) // jackson-core is provided by :libs:x-content:impl. If declared here, there's a module issue that prevents ES from starting: // @@ -29,18 +29,18 @@ dependencies { // at org.elasticsearch.xcontent@9.0.0-SNAPSHOT/org.elasticsearch.xcontent.json.JsonXContent.(JsonXContent.java:37) // at org.elasticsearch.xcontent@9.0.0-SNAPSHOT/org.elasticsearch.xcontent.XContentType.(XContentType.java:28) // at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.common.settings.Setting.arrayToParsableString(Setting.java:1883) - implementation(project(":libs:x-content:impl")) + //implementation(project(":libs:x-content:impl")) // arrow-vector api("org.apache.arrow:arrow-vector:${arrowVersion}") - //api("com.fasterxml.jackson.core:jackson-core:${versions.jackson}") + api("com.fasterxml.jackson.core:jackson-core:${versions.jackson}") api("com.fasterxml.jackson.core:jackson-annotations:${versions.jackson}") api("com.fasterxml.jackson.core:jackson-databind:${versions.jackson}") api("com.fasterxml.jackson.datatype:jackson-datatype-jsr310:${versions.jackson}") api("com.google.flatbuffers:flatbuffers-java:24.3.25") api("commons-codec:commons-codec:${versions.commonscodec}") // Arrow 18 -> commons-codec 1.17.1 - api("org.slf4j:slf4j-api:2.0.10") + api("org.slf4j:slf4j-api:${versions.slf4j}") api("org.immutables:value-annotations:2.10.1") // provided dependency // arrow-format @@ -54,7 +54,7 @@ dependencies { // also depends on value-annotations (provided dependency) // arrow-memory-unsafe - implementation("org.apache.arrow:arrow-memory-unsafe:${arrowVersion}") + api("org.apache.arrow:arrow-memory-unsafe:${arrowVersion}") // also depends on value-annotations (provided dependency) testImplementation(project(":test:framework")) { diff --git a/libs/arrow/licenses/commons-codec-NOTICE.txt b/libs/arrow/licenses/commons-codec-NOTICE.txt index 1da9af50f6008..56916449bbe10 100644 --- a/libs/arrow/licenses/commons-codec-NOTICE.txt +++ b/libs/arrow/licenses/commons-codec-NOTICE.txt @@ -1,5 +1,5 @@ Apache Commons Codec -Copyright 2002-2014 The Apache Software Foundation +Copyright 2002-2015 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/libs/arrow/src/main/java/module-info.java-disabled b/libs/arrow/src/main/java/module-info.java-disabled new file mode 100644 index 0000000000000..33b421033f218 --- /dev/null +++ b/libs/arrow/src/main/java/module-info.java-disabled @@ -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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +// Note: we cannot use Java modules for this library. +// +// For performance reasons Arrow accesses some private fields of ByteBuffer, and this requires "opens" +// permission on the java.base/java.nio package. See https://arrow.apache.org/docs/java/install.html +// +// - if this module is loaded from a plugin, the JVM will not know it at startup, and will not honor +// --add-opens=java.base/java.nio=org.apache.arrow.memory.core. +// The startup logs will contain: "Unknown module: org.apache.arrow.memory.core specified to --add-opens" +// +// - if loaded from the server or core modules, it reveals versions conflicts: +// - because of xcontent:impl com.fasterxml.jackson.core:jackson-core between versions 2.17.2 and 2.15.0 +// - because of ?? commons-codec:commons-codec between versions 1.16.1 and 1.15 +// Fixing them isn't a trivial task (also impacts serverless). +// +// So we have to disable Java modules for this Gradle module and use -add-opens=java.base/ALL-UNNAMED +// in distribution/src/config/jvm.options until the version conflict issue is solved and we can load it from +// the server or core modules. + +module org.elasticsearch.libs.arrow { + exports org.elasticsearch.libs.arrow; + + requires transitive org.apache.arrow.memory.core; + requires transitive org.apache.arrow.vector; + requires transitive org.apache.arrow.format; + requires org.apache.arrow.memory.unsafe; +} diff --git a/libs/arrow/src/main/java/org/elasticsearch/arrow/Arrow.java b/libs/arrow/src/main/java/org/elasticsearch/libs/arrow/Arrow.java similarity index 53% rename from libs/arrow/src/main/java/org/elasticsearch/arrow/Arrow.java rename to libs/arrow/src/main/java/org/elasticsearch/libs/arrow/Arrow.java index e5764da5d3e3b..d3f306fc64acc 100644 --- a/libs/arrow/src/main/java/org/elasticsearch/arrow/Arrow.java +++ b/libs/arrow/src/main/java/org/elasticsearch/libs/arrow/Arrow.java @@ -7,8 +7,10 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ -package org.elasticsearch.arrow; +package org.elasticsearch.libs.arrow; +import org.apache.arrow.memory.AllocationListener; +import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.memory.RootAllocator; public class Arrow { @@ -24,10 +26,24 @@ public class Arrow { private static final RootAllocator ROOT_ALLOCATOR = new RootAllocator(); /** - * Returns the global root allocator. Should be used to create child allocators to have - * fine-grained memory allocation tracking and to enforce local limits. + * Returns the global root allocator. Do not use it to allocate memory, use {@link #newChildAllocator(String, long, long)} to + * enforce allocation limits and track potential memory leaks when the child allocator is closed. */ public static RootAllocator rootAllocator() { return ROOT_ALLOCATOR; } + + /** + * Creates a new allocator, child of the root allocator. + */ + public static BufferAllocator newChildAllocator(String name, long initReservation, long maxAllocation) { + return ROOT_ALLOCATOR.newChildAllocator(name, initReservation, maxAllocation); + } + + /** + * Creates a new allocator, child of the root allocator. + */ + public static BufferAllocator newChildAllocator(String name, AllocationListener listener, long initReservation, long maxAllocation) { + return ROOT_ALLOCATOR.newChildAllocator(name, listener, initReservation, maxAllocation); + } } diff --git a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowFormatException.java b/libs/arrow/src/main/java/org/elasticsearch/libs/arrow/ArrowFormatException.java similarity index 94% rename from libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowFormatException.java rename to libs/arrow/src/main/java/org/elasticsearch/libs/arrow/ArrowFormatException.java index c0068ff7d9578..bc350c9061258 100644 --- a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowFormatException.java +++ b/libs/arrow/src/main/java/org/elasticsearch/libs/arrow/ArrowFormatException.java @@ -7,7 +7,7 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ -package org.elasticsearch.arrow; +package org.elasticsearch.libs.arrow; public class ArrowFormatException extends RuntimeException { diff --git a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java index d25f76423bceb..682bdad3d7c0b 100644 --- a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java +++ b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/initialization/EntitlementInitialization.java @@ -270,20 +270,6 @@ private static PolicyManager createPolicyManager() { ); } - // In modules:repository-s3 and plugins:discovery-ec2, aws-java-sdk-core reads its default configuration from a json file in - // its jar. Its URL is opened by Jackson, causing an entitlement failure in `(server)`. - Collections.addAll( - serverScopes, - new Scope( - "com.fasterxml.jackson.core", - List.of( - new FilesEntitlement( - List.of(FileData.ofPath(bootstrapArgs.modulesDir(), READ), FileData.ofPath(bootstrapArgs.pluginsDir(), READ)) - ) - ) - ) - ); - var serverPolicy = new Policy( "server", bootstrapArgs.serverPolicyPatch() == null diff --git a/libs/x-content/impl/build.gradle b/libs/x-content/impl/build.gradle index 06928662170d0..35e122d336c68 100644 --- a/libs/x-content/impl/build.gradle +++ b/libs/x-content/impl/build.gradle @@ -13,13 +13,15 @@ base { archivesName = "x-content-impl" } +String jacksonVersion = "2.17.2" + dependencies { compileOnly project(':libs:core') compileOnly project(':libs:x-content') - api "com.fasterxml.jackson.core:jackson-core:${versions.jackson}" - implementation "com.fasterxml.jackson.dataformat:jackson-dataformat-smile:${versions.jackson}" - implementation "com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:${versions.jackson}" - implementation "com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:${versions.jackson}" + implementation "com.fasterxml.jackson.core:jackson-core:${jacksonVersion}" + implementation "com.fasterxml.jackson.dataformat:jackson-dataformat-smile:${jacksonVersion}" + implementation "com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:${jacksonVersion}" + implementation "com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:${jacksonVersion}" implementation "org.yaml:snakeyaml:${versions.snakeyaml}" testImplementation(project(":test:framework")) { diff --git a/libs/x-content/impl/src/main/java/module-info.java b/libs/x-content/impl/src/main/java/module-info.java index cb0f285e09ddb..61ca74e4effd2 100644 --- a/libs/x-content/impl/src/main/java/module-info.java +++ b/libs/x-content/impl/src/main/java/module-info.java @@ -8,7 +8,7 @@ */ module org.elasticsearch.xcontent.impl { - requires transitive com.fasterxml.jackson.core; + requires com.fasterxml.jackson.core; requires com.fasterxml.jackson.dataformat.cbor; requires com.fasterxml.jackson.dataformat.smile; requires com.fasterxml.jackson.dataformat.yaml; diff --git a/modules/arrow/build.gradle b/modules/arrow/build.gradle new file mode 100644 index 0000000000000..ae394fef4512f --- /dev/null +++ b/modules/arrow/build.gradle @@ -0,0 +1,34 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +import org.elasticsearch.gradle.internal.precommit.CheckForbiddenApisTask + +apply plugin: 'elasticsearch.internal-es-plugin' +apply plugin: 'elasticsearch.internal-java-rest-test' + +esplugin { + name = 'arrow' + description = 'Provides Arrow integration for Elasticsearch' + classname ='org.elasticsearch.arrow.ArrowPlugin' + //extendedPlugins = ['lang-painless'] +} + +dependencies { + implementation(project(":libs:x-content")) + implementation(project(":libs:arrow")) + implementation("com.fasterxml.jackson.core:jackson-core:${versions.jackson}") + implementation("com.fasterxml.jackson.core:jackson-databind:${versions.jackson}") + implementation("com.fasterxml.jackson.core:jackson-annotations:${versions.jackson}") + + testImplementation(project(":test:framework")) +} + +tasks.named("dependencyLicenses").configure { + mapping from: /jackson-.*/, to: 'jackson' +} diff --git a/modules/arrow/licenses/jackson-LICENSE.txt b/modules/arrow/licenses/jackson-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/modules/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. diff --git a/modules/arrow/licenses/jackson-NOTICE.txt b/modules/arrow/licenses/jackson-NOTICE.txt new file mode 100644 index 0000000000000..4c976b7b4cc58 --- /dev/null +++ b/modules/arrow/licenses/jackson-NOTICE.txt @@ -0,0 +1,20 @@ +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. diff --git a/libs/arrow/src/main/java/module-info.java b/modules/arrow/src/main/java/module-info.java-disabled similarity index 62% rename from libs/arrow/src/main/java/module-info.java rename to modules/arrow/src/main/java/module-info.java-disabled index f6c0d3e1976dd..a3b51edc7b9ed 100644 --- a/libs/arrow/src/main/java/module-info.java +++ b/modules/arrow/src/main/java/module-info.java-disabled @@ -7,14 +7,18 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ -module org.elasticsearch.arrow { - exports org.elasticsearch.arrow; +// See why Java modules are disabled here in libs/arrow/src/main/java/module-info.java-disabled - requires transitive org.apache.arrow.vector; - requires transitive org.apache.arrow.memory.core; - requires transitive org.apache.arrow.format; +module org.elasticsearch.arrow { + requires org.apache.arrow.vector; + requires org.apache.arrow.memory.core; + requires org.apache.arrow.format; - requires org.elasticsearch.xcontent; + requires com.fasterxml.jackson.core; requires com.fasterxml.jackson.databind; + requires org.elasticsearch.libs.arrow; + requires org.elasticsearch.xcontent; + requires org.elasticsearch.server; requires org.elasticsearch.base; + requires org.apache.lucene.core; } diff --git a/modules/arrow/src/main/java/org/elasticsearch/arrow/ArrowPlugin.java b/modules/arrow/src/main/java/org/elasticsearch/arrow/ArrowPlugin.java new file mode 100644 index 0000000000000..3492d43b2ee9e --- /dev/null +++ b/modules/arrow/src/main/java/org/elasticsearch/arrow/ArrowPlugin.java @@ -0,0 +1,74 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.arrow; + +import org.elasticsearch.arrow.bulk.ArrowBulkAction; +import org.elasticsearch.client.internal.Client; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.IndexScopedSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.settings.SettingsFilter; +import org.elasticsearch.features.NodeFeature; +import org.elasticsearch.libs.arrow.Arrow; +import org.elasticsearch.plugins.ActionPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestHandler; + +import java.util.Collection; +import java.util.List; +import java.util.function.Predicate; +import java.util.function.Supplier; + +public class ArrowPlugin extends Plugin implements ActionPlugin { + + private Client client; + + @Override + public Collection createComponents(PluginServices services) { + this.client = services.client(); + + return List.of(new AbstractLifecycleComponent() { + @Override + protected void doStart() { + // Make sure Arrow is initialized + Arrow.rootAllocator(); + } + + @Override + protected void doStop() {} + + @Override + protected void doClose() {} + }); + } + + /** + * Rest handlers added by this plugin. + */ + @Override + public Collection getRestHandlers( + Settings settings, + NamedWriteableRegistry namedWriteableRegistry, + RestController restController, + ClusterSettings clusterSettings, + IndexScopedSettings indexScopedSettings, + SettingsFilter settingsFilter, + IndexNameExpressionResolver indexNameExpressionResolver, + Supplier nodesInCluster, + Predicate clusterSupportsFeature + ) { + return List.of(new ArrowBulkAction(this.client, settings)); + } +} diff --git a/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkAction.java b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkAction.java new file mode 100644 index 0000000000000..f6b557a70c7ae --- /dev/null +++ b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkAction.java @@ -0,0 +1,66 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.arrow.bulk; + +import org.elasticsearch.action.bulk.BulkShardRequest; +import org.elasticsearch.action.bulk.IncrementalBulkService; +import org.elasticsearch.client.internal.Client; +import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.index.IndexingPressure; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.action.document.RestBulkAction; + +import java.io.IOException; +import java.util.List; + +import static org.elasticsearch.rest.RestRequest.Method.POST; +import static org.elasticsearch.rest.RestRequest.Method.PUT; + +public class ArrowBulkAction extends BaseRestHandler { + + private final IncrementalBulkService bulkHandler; + + public ArrowBulkAction(Client client, Settings settings) { + this.bulkHandler = new IncrementalBulkService(client, new IndexingPressure(settings)); + } + + @Override + public String getName() { + return "arrow_bulk_action"; + } + + @Override + public List routes() { + return List.of( + new Route(POST, "/_arrow/_bulk"), + new Route(PUT, "/_arrow/_bulk"), + new Route(POST, "/_arrow/{index}/_bulk"), + new Route(PUT, "/_arrow/{index}/_bulk") + ); + } + + public boolean mediaTypesValid(RestRequest request) { + return ArrowBulkRequestParser.isArrowRequest(request); + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException { + String waitForActiveShards = request.param("wait_for_active_shards"); + TimeValue timeout = request.paramAsTime("timeout", BulkShardRequest.DEFAULT_TIMEOUT); + String refresh = request.param("refresh"); + return new RestBulkAction.ChunkHandler(false, request, + () -> bulkHandler.newBulkRequest(waitForActiveShards, timeout, refresh), + new ArrowBulkRequestParser(request) + ); + } +} diff --git a/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParser.java b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkIncrementalParser.java similarity index 97% rename from server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParser.java rename to modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkIncrementalParser.java index 6651803fe50d3..b5f34df43b6c6 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParser.java +++ b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkIncrementalParser.java @@ -7,7 +7,7 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ -package org.elasticsearch.action.bulk.arrow; +package org.elasticsearch.arrow.bulk; import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.memory.RootAllocator; @@ -28,13 +28,13 @@ import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.update.UpdateRequest; -import org.elasticsearch.arrow.Arrow; -import org.elasticsearch.arrow.ArrowFormatException; -import org.elasticsearch.arrow.ArrowToXContent; -import org.elasticsearch.arrow.XContentBuffer; +import org.elasticsearch.arrow.xcontent.ArrowToXContent; +import org.elasticsearch.arrow.xcontent.XContentBuffer; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.core.Nullable; +import org.elasticsearch.libs.arrow.Arrow; +import org.elasticsearch.libs.arrow.ArrowFormatException; import org.elasticsearch.search.fetch.subphase.FetchSourceContext; import org.elasticsearch.xcontent.XContent; import org.elasticsearch.xcontent.XContentParserConfiguration; @@ -106,7 +106,7 @@ class ArrowBulkIncrementalParser extends BulkRequestParser.XContentIncrementalPa this.defaultOpType = defaultOpType; // FIXME: hard-coded limit to 100 MiB per record batch. Should we add an AllocationListener that calls ES memory management? - this.allocator = Arrow.rootAllocator().newChildAllocator("bulk-ingestion", 0, 100 * 1024 * 1024); + this.allocator = Arrow.newChildAllocator("bulk-ingestion", 0, 100 * 1024 * 1024); this.arrowParser = new ArrowIncrementalParser(new RootAllocator(), new ArrowIncrementalParser.Listener() { @Override diff --git a/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkRequestParser.java b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkRequestParser.java similarity index 91% rename from server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkRequestParser.java rename to modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkRequestParser.java index ed3e20b79e074..a9079b547a54e 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowBulkRequestParser.java +++ b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkRequestParser.java @@ -7,17 +7,17 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ -package org.elasticsearch.action.bulk.arrow; +package org.elasticsearch.arrow.bulk; import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.bulk.AbstractBulkRequestParser; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.update.UpdateRequest; -import org.elasticsearch.arrow.Arrow; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.RestApiVersion; +import org.elasticsearch.libs.arrow.Arrow; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.search.fetch.subphase.FetchSourceContext; import org.elasticsearch.xcontent.XContentParserConfiguration; @@ -30,14 +30,7 @@ public class ArrowBulkRequestParser extends AbstractBulkRequestParser { public static boolean isArrowRequest(RestRequest request) { - if (request.getXContentType() != null) { - return false; // We don't have a XContentType for Arrow (yet) - } - var contentType = request.getHeaders().get("Content-Type"); - if (contentType == null || contentType.isEmpty()) { - return false; - } - return contentType.contains(Arrow.MEDIA_TYPE); + return request.getParsedContentType().mediaTypeWithoutParameters().equals(Arrow.MEDIA_TYPE); } private final RestApiVersion apiVersion; diff --git a/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowIncrementalParser.java b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowIncrementalParser.java similarity index 95% rename from server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowIncrementalParser.java rename to modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowIncrementalParser.java index 81a9d73c958eb..ef626e96cd30d 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/arrow/ArrowIncrementalParser.java +++ b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowIncrementalParser.java @@ -7,7 +7,7 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ -package org.elasticsearch.action.bulk.arrow; +package org.elasticsearch.arrow.bulk; import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.vector.VectorSchemaRoot; @@ -16,6 +16,7 @@ import org.apache.arrow.vector.ipc.ReadChannel; import org.apache.arrow.vector.ipc.message.MessageMetadataResult; import org.apache.arrow.vector.ipc.message.MessageSerializer; +import org.elasticsearch.libs.arrow.ArrowFormatException; import org.elasticsearch.common.bytes.BytesReference; import java.io.Closeable; @@ -81,12 +82,17 @@ public void close() throws IOException { public int parse(BytesReference data, boolean lastData) throws IOException { int total = 0; int consumed; + int chunkLength = data.length(); while ((consumed = doParse(data, lastData)) > 0) { total += consumed; data = data.slice(consumed, data.length() - consumed); // Start a new message expectedDataLength = PREFIX_LEN; } + + if (lastData && total != chunkLength) { + throw new ArrowFormatException("Incomplete or invalid Arrow stream"); + } return total; } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannel.java b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/BytesReferenceChannel.java similarity index 98% rename from server/src/main/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannel.java rename to modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/BytesReferenceChannel.java index 49eaacb3eff75..b60322cd6df93 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannel.java +++ b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/BytesReferenceChannel.java @@ -7,7 +7,7 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ -package org.elasticsearch.action.bulk.arrow; +package org.elasticsearch.arrow.bulk; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; diff --git a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonParser.java b/modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent/ArrowJsonParser.java similarity index 98% rename from libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonParser.java rename to modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent/ArrowJsonParser.java index 0487bd75ab4c1..d75ddd5e1c1f5 100644 --- a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonParser.java +++ b/modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent/ArrowJsonParser.java @@ -7,7 +7,7 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ -package org.elasticsearch.arrow; +package org.elasticsearch.arrow.xcontent; import com.fasterxml.jackson.core.JsonFactory; import com.fasterxml.jackson.core.JsonGenerator; @@ -32,6 +32,7 @@ import org.apache.arrow.vector.complex.UnionVector; import org.apache.arrow.vector.ipc.ArrowStreamReader; import org.apache.arrow.vector.types.Types; +import org.elasticsearch.libs.arrow.ArrowFormatException; import java.io.IOException; import java.io.InputStream; diff --git a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonXContentParser.java b/modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent/ArrowJsonXContentParser.java similarity index 99% rename from libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonXContentParser.java rename to modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent/ArrowJsonXContentParser.java index e61d25dd5f795..f2b7aedf931f5 100644 --- a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowJsonXContentParser.java +++ b/modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent/ArrowJsonXContentParser.java @@ -7,7 +7,7 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ -package org.elasticsearch.arrow; +package org.elasticsearch.arrow.xcontent; import com.fasterxml.jackson.core.JsonLocation; import com.fasterxml.jackson.core.JsonParseException; diff --git a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowToXContent.java b/modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent/ArrowToXContent.java similarity index 99% rename from libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowToXContent.java rename to modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent/ArrowToXContent.java index 6fc4f32b9c7a3..950cf5e3ef0bb 100644 --- a/libs/arrow/src/main/java/org/elasticsearch/arrow/ArrowToXContent.java +++ b/modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent/ArrowToXContent.java @@ -7,7 +7,7 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ -package org.elasticsearch.arrow; +package org.elasticsearch.arrow.xcontent; import org.apache.arrow.vector.BaseIntVector; import org.apache.arrow.vector.BitVector; @@ -24,6 +24,7 @@ import org.apache.arrow.vector.complex.UnionVector; import org.apache.arrow.vector.dictionary.Dictionary; import org.apache.arrow.vector.types.Types; +import org.elasticsearch.libs.arrow.ArrowFormatException; import org.elasticsearch.xcontent.XContentGenerator; import java.io.IOException; diff --git a/libs/arrow/src/main/java/org/elasticsearch/arrow/XContentBuffer.java b/modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent/XContentBuffer.java similarity index 99% rename from libs/arrow/src/main/java/org/elasticsearch/arrow/XContentBuffer.java rename to modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent/XContentBuffer.java index 16f00a485b29d..502c6619e77c7 100644 --- a/libs/arrow/src/main/java/org/elasticsearch/arrow/XContentBuffer.java +++ b/modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent/XContentBuffer.java @@ -7,7 +7,7 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ -package org.elasticsearch.arrow; +package org.elasticsearch.arrow.xcontent; import com.fasterxml.jackson.core.JsonGenerationException; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/libs/arrow/src/test/java/org/elasticsearch/arrow/ArrowToXContentTests.java b/modules/arrow/src/test/java/org/elasticsearch/arrow/xcontent/ArrowToXContentTests.java similarity index 93% rename from libs/arrow/src/test/java/org/elasticsearch/arrow/ArrowToXContentTests.java rename to modules/arrow/src/test/java/org/elasticsearch/arrow/xcontent/ArrowToXContentTests.java index 3207ff1cd9efa..8632cef233dc1 100644 --- a/libs/arrow/src/test/java/org/elasticsearch/arrow/ArrowToXContentTests.java +++ b/modules/arrow/src/test/java/org/elasticsearch/arrow/xcontent/ArrowToXContentTests.java @@ -7,13 +7,14 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ -package org.elasticsearch.arrow; +package org.elasticsearch.arrow.xcontent; import org.apache.arrow.vector.IntVector; import org.apache.arrow.vector.NullVector; import org.apache.arrow.vector.ValueVector; import org.apache.arrow.vector.VarCharVector; import org.apache.arrow.vector.complex.MapVector; +import org.elasticsearch.libs.arrow.Arrow; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xcontent.XContentType; @@ -37,8 +38,8 @@ private static void checkPosition(ValueVector vector, int position, String json) public void testWriteField() throws IOException { try ( - var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); - IntVector vector = new IntVector("intField", allocator); + var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); + IntVector vector = new IntVector("intField", allocator); ) { vector.allocateNew(1); vector.set(0, 123); diff --git a/modules/ingest-attachment/build.gradle b/modules/ingest-attachment/build.gradle index 388b11cef0f15..a172112948fd3 100644 --- a/modules/ingest-attachment/build.gradle +++ b/modules/ingest-attachment/build.gradle @@ -18,14 +18,14 @@ esplugin { // as we (and tika) demand, and are not interested in, say, having the same version of commons-codec as elasticsearch itself // when updating tika, please review it's parent pom : https://repo1.maven.org/maven2/org/apache/tika/tika-parent // and manually update the transitive dependencies here -def buildVersions = [ +def versions = [ 'tika' : '2.9.2', 'pdfbox': '2.0.31', 'poi' : '5.2.5', 'sparsebitset' : '1.3', //poi dependency: https://repo1.maven.org/maven2/org/apache/poi/poi/ 'mime4j': '0.8.11', - 'commonsCodec': versions.commonscodec, - 'slf4' : versions.slf4j, + 'commonsCodec': '1.16.1', + 'slf4' : '2.0.10', 'xz' : '1.9', 'commonsIo' : '2.15.1', //intentionally using the elder "juniversalchardet:juniversalchardet" rather than the newer "com.github.albfernandez:juniversalchardet" @@ -44,18 +44,18 @@ def buildVersions = [ configurations.testRuntimeClasspath { exclude module: 'commons-logging' // The version used by POI potentially conflicts with the one pulled in by :test:framework: - resolutionStrategy.force "commons-codec:commons-codec:${buildVersions.commonsCodec}" + resolutionStrategy.force "commons-codec:commons-codec:${versions.commonsCodec}" } configurations.testCompileClasspath { // The version used by POI potentially conflicts with the one pulled in by :test:framework: - resolutionStrategy.force "commons-codec:commons-codec:${buildVersions.commonsCodec}" + resolutionStrategy.force "commons-codec:commons-codec:${versions.commonsCodec}" } dependencies { // take over logging for all dependencies - api "org.slf4j:slf4j-api:${buildVersions.slf4}" - api "org.slf4j:jcl-over-slf4j:${buildVersions.slf4}" + api "org.slf4j:slf4j-api:${versions.slf4}" + api "org.slf4j:jcl-over-slf4j:${versions.slf4}" // route slf4j over log4j // TODO blocked on https://github.com/elastic/elasticsearch/issues/93714 @@ -63,54 +63,54 @@ dependencies { // nop all slf4j logging // workaround for https://github.com/elastic/elasticsearch/issues/93714 - api "org.slf4j:slf4j-nop:${buildVersions.slf4}" + api "org.slf4j:slf4j-nop:${versions.slf4}" // mandatory for tika - api "org.apache.tika:tika-core:${buildVersions.tika}" - api "org.apache.tika:tika-langdetect:${buildVersions.tika}" - api "org.apache.tika:tika-langdetect-tika:${buildVersions.tika}" - api "org.apache.tika:tika-parser-html-module:${buildVersions.tika}" - api "org.apache.tika:tika-parser-microsoft-module:${buildVersions.tika}" - api "org.apache.tika:tika-parser-pdf-module:${buildVersions.tika}" - api "org.apache.tika:tika-parser-xml-module:${buildVersions.tika}" - api "org.apache.tika:tika-parser-text-module:${buildVersions.tika}" - api "org.apache.tika:tika-parser-miscoffice-module:${buildVersions.tika}" - api "org.apache.tika:tika-parser-apple-module:${buildVersions.tika}" - api "org.apache.tika:tika-parser-xmp-commons:${buildVersions.tika}" - api "org.apache.tika:tika-parser-zip-commons:${buildVersions.tika}" - api "org.tukaani:xz:${buildVersions.xz}" - api "commons-io:commons-io:${buildVersions.commonsIo}" + api "org.apache.tika:tika-core:${versions.tika}" + api "org.apache.tika:tika-langdetect:${versions.tika}" + api "org.apache.tika:tika-langdetect-tika:${versions.tika}" + api "org.apache.tika:tika-parser-html-module:${versions.tika}" + api "org.apache.tika:tika-parser-microsoft-module:${versions.tika}" + api "org.apache.tika:tika-parser-pdf-module:${versions.tika}" + api "org.apache.tika:tika-parser-xml-module:${versions.tika}" + api "org.apache.tika:tika-parser-text-module:${versions.tika}" + api "org.apache.tika:tika-parser-miscoffice-module:${versions.tika}" + api "org.apache.tika:tika-parser-apple-module:${versions.tika}" + api "org.apache.tika:tika-parser-xmp-commons:${versions.tika}" + api "org.apache.tika:tika-parser-zip-commons:${versions.tika}" + api "org.tukaani:xz:${versions.xz}" + api "commons-io:commons-io:${versions.commonsIo}" // character set detection - api "com.googlecode.juniversalchardet:juniversalchardet:${buildVersions.juniversalchardet}" + api "com.googlecode.juniversalchardet:juniversalchardet:${versions.juniversalchardet}" // external parser libraries // HTML - api "org.ccil.cowan.tagsoup:tagsoup:${buildVersions.tagsoup}" + api "org.ccil.cowan.tagsoup:tagsoup:${versions.tagsoup}" // Adobe PDF - api "org.apache.pdfbox:pdfbox:${buildVersions.pdfbox}" - api "org.apache.pdfbox:fontbox:${buildVersions.pdfbox}" - api "org.apache.pdfbox:jempbox:${buildVersions.jempbox}" + api "org.apache.pdfbox:pdfbox:${versions.pdfbox}" + api "org.apache.pdfbox:fontbox:${versions.pdfbox}" + api "org.apache.pdfbox:jempbox:${versions.jempbox}" // OpenOffice - api "org.apache.poi:poi-ooxml:${buildVersions.poi}" - api "org.apache.poi:poi-ooxml-lite:${buildVersions.poi}" - api "org.apache.poi:poi:${buildVersions.poi}" - api "commons-codec:commons-codec:${buildVersions.commonsCodec}" - api "org.apache.xmlbeans:xmlbeans:${buildVersions.xmlbeans}" - api "org.apache.commons:commons-collections4:${buildVersions.commonsCollections4}" + api "org.apache.poi:poi-ooxml:${versions.poi}" + api "org.apache.poi:poi-ooxml-lite:${versions.poi}" + api "org.apache.poi:poi:${versions.poi}" + api "commons-codec:commons-codec:${versions.commonsCodec}" + api "org.apache.xmlbeans:xmlbeans:${versions.xmlbeans}" + api "org.apache.commons:commons-collections4:${versions.commonsCollections4}" // MS Office - api "org.apache.poi:poi-scratchpad:${buildVersions.poi}" + api "org.apache.poi:poi-scratchpad:${versions.poi}" // Apple iWork - api "org.apache.commons:commons-compress:${buildVersions.commonsCompress}" + api "org.apache.commons:commons-compress:${versions.commonsCompress}" // Outlook documents - api "org.apache.james:apache-mime4j-core:${buildVersions.mime4j}" - api "org.apache.james:apache-mime4j-dom:${buildVersions.mime4j}" + api "org.apache.james:apache-mime4j-core:${versions.mime4j}" + api "org.apache.james:apache-mime4j-dom:${versions.mime4j}" // EPUB books - api "org.apache.commons:commons-lang3:${buildVersions.commonsLang3}" + api "org.apache.commons:commons-lang3:${versions.commonsLang3}" // Microsoft Word files with visio diagrams - api "org.apache.commons:commons-math3:${buildVersions.commonsMath3}" + api "org.apache.commons:commons-math3:${versions.commonsMath3}" // POIs dependency - api "com.zaxxer:SparseBitSet:${buildVersions.sparsebitset}" + api "com.zaxxer:SparseBitSet:${versions.sparsebitset}" } restResources { diff --git a/modules/ingest-attachment/licenses/commons-codec-LICENSE.txt b/modules/ingest-attachment/licenses/commons-codec-LICENSE.txt new file mode 100644 index 0000000000000..57bc88a15a0ee --- /dev/null +++ b/modules/ingest-attachment/licenses/commons-codec-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. + diff --git a/modules/ingest-attachment/licenses/commons-codec-NOTICE.txt b/modules/ingest-attachment/licenses/commons-codec-NOTICE.txt new file mode 100644 index 0000000000000..56916449bbe10 --- /dev/null +++ b/modules/ingest-attachment/licenses/commons-codec-NOTICE.txt @@ -0,0 +1,17 @@ +Apache Commons Codec +Copyright 2002-2015 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java +contains test data from http://aspell.net/test/orig/batch0.tab. +Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + +=============================================================================== + +The content of package org.apache.commons.codec.language.bm has been translated +from the original php source code available at http://stevemorse.org/phoneticinfo.htm +with permission from the original authors. +Original source copyright: +Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/modules/ingest-attachment/licenses/slf4j-api-LICENSE.txt b/modules/ingest-attachment/licenses/slf4j-api-LICENSE.txt new file mode 100644 index 0000000000000..52055e61de46f --- /dev/null +++ b/modules/ingest-attachment/licenses/slf4j-api-LICENSE.txt @@ -0,0 +1,21 @@ +Copyright (c) 2004-2014 QOS.ch +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. diff --git a/modules/ingest-attachment/licenses/slf4j-api-NOTICE.txt b/modules/ingest-attachment/licenses/slf4j-api-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/modules/ingest-geoip/licenses/jackson-LICENSE.txt b/modules/ingest-geoip/licenses/jackson-LICENSE.txt new file mode 100644 index 0000000000000..f5f45d26a49d6 --- /dev/null +++ b/modules/ingest-geoip/licenses/jackson-LICENSE.txt @@ -0,0 +1,8 @@ +This copy of Jackson JSON processor streaming parser/generator is licensed under the +Apache (Software) License, version 2.0 ("the License"). +See the License for details about distribution rights, and the +specific rights regarding derivate works. + +You may obtain a copy of the License at: + +http://www.apache.org/licenses/LICENSE-2.0 diff --git a/modules/ingest-geoip/licenses/jackson-NOTICE.txt b/modules/ingest-geoip/licenses/jackson-NOTICE.txt new file mode 100644 index 0000000000000..4c976b7b4cc58 --- /dev/null +++ b/modules/ingest-geoip/licenses/jackson-NOTICE.txt @@ -0,0 +1,20 @@ +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. diff --git a/modules/legacy-geo/licenses/jackson-LICENSE.txt b/modules/legacy-geo/licenses/jackson-LICENSE.txt new file mode 100644 index 0000000000000..f5f45d26a49d6 --- /dev/null +++ b/modules/legacy-geo/licenses/jackson-LICENSE.txt @@ -0,0 +1,8 @@ +This copy of Jackson JSON processor streaming parser/generator is licensed under the +Apache (Software) License, version 2.0 ("the License"). +See the License for details about distribution rights, and the +specific rights regarding derivate works. + +You may obtain a copy of the License at: + +http://www.apache.org/licenses/LICENSE-2.0 diff --git a/modules/legacy-geo/licenses/jackson-NOTICE.txt b/modules/legacy-geo/licenses/jackson-NOTICE.txt new file mode 100644 index 0000000000000..4c976b7b4cc58 --- /dev/null +++ b/modules/legacy-geo/licenses/jackson-NOTICE.txt @@ -0,0 +1,20 @@ +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. diff --git a/modules/repository-azure/licenses/slf4j-api-LICENSE.txt b/modules/repository-azure/licenses/slf4j-api-LICENSE.txt new file mode 100644 index 0000000000000..52055e61de46f --- /dev/null +++ b/modules/repository-azure/licenses/slf4j-api-LICENSE.txt @@ -0,0 +1,21 @@ +Copyright (c) 2004-2014 QOS.ch +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. diff --git a/modules/repository-azure/licenses/slf4j-api-NOTICE.txt b/modules/repository-azure/licenses/slf4j-api-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/modules/repository-gcs/licenses/commons-codec-LICENSE.txt b/modules/repository-gcs/licenses/commons-codec-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/modules/repository-gcs/licenses/commons-codec-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. diff --git a/modules/repository-gcs/licenses/commons-codec-NOTICE.txt b/modules/repository-gcs/licenses/commons-codec-NOTICE.txt new file mode 100644 index 0000000000000..56916449bbe10 --- /dev/null +++ b/modules/repository-gcs/licenses/commons-codec-NOTICE.txt @@ -0,0 +1,17 @@ +Apache Commons Codec +Copyright 2002-2015 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java +contains test data from http://aspell.net/test/orig/batch0.tab. +Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + +=============================================================================== + +The content of package org.apache.commons.codec.language.bm has been translated +from the original php source code available at http://stevemorse.org/phoneticinfo.htm +with permission from the original authors. +Original source copyright: +Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/modules/repository-gcs/licenses/jackson-LICENSE.txt b/modules/repository-gcs/licenses/jackson-LICENSE.txt new file mode 100644 index 0000000000000..f5f45d26a49d6 --- /dev/null +++ b/modules/repository-gcs/licenses/jackson-LICENSE.txt @@ -0,0 +1,8 @@ +This copy of Jackson JSON processor streaming parser/generator is licensed under the +Apache (Software) License, version 2.0 ("the License"). +See the License for details about distribution rights, and the +specific rights regarding derivate works. + +You may obtain a copy of the License at: + +http://www.apache.org/licenses/LICENSE-2.0 diff --git a/modules/repository-gcs/licenses/jackson-NOTICE.txt b/modules/repository-gcs/licenses/jackson-NOTICE.txt new file mode 100644 index 0000000000000..4c976b7b4cc58 --- /dev/null +++ b/modules/repository-gcs/licenses/jackson-NOTICE.txt @@ -0,0 +1,20 @@ +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. diff --git a/modules/repository-s3/licenses/commons-codec-LICENSE.txt b/modules/repository-s3/licenses/commons-codec-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/modules/repository-s3/licenses/commons-codec-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. diff --git a/modules/repository-s3/licenses/commons-codec-NOTICE.txt b/modules/repository-s3/licenses/commons-codec-NOTICE.txt new file mode 100644 index 0000000000000..56916449bbe10 --- /dev/null +++ b/modules/repository-s3/licenses/commons-codec-NOTICE.txt @@ -0,0 +1,17 @@ +Apache Commons Codec +Copyright 2002-2015 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java +contains test data from http://aspell.net/test/orig/batch0.tab. +Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + +=============================================================================== + +The content of package org.apache.commons.codec.language.bm has been translated +from the original php source code available at http://stevemorse.org/phoneticinfo.htm +with permission from the original authors. +Original source copyright: +Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/modules/repository-s3/licenses/jackson-LICENSE b/modules/repository-s3/licenses/jackson-LICENSE new file mode 100644 index 0000000000000..f5f45d26a49d6 --- /dev/null +++ b/modules/repository-s3/licenses/jackson-LICENSE @@ -0,0 +1,8 @@ +This copy of Jackson JSON processor streaming parser/generator is licensed under the +Apache (Software) License, version 2.0 ("the License"). +See the License for details about distribution rights, and the +specific rights regarding derivate works. + +You may obtain a copy of the License at: + +http://www.apache.org/licenses/LICENSE-2.0 diff --git a/modules/repository-s3/licenses/jackson-NOTICE b/modules/repository-s3/licenses/jackson-NOTICE new file mode 100644 index 0000000000000..4c976b7b4cc58 --- /dev/null +++ b/modules/repository-s3/licenses/jackson-NOTICE @@ -0,0 +1,20 @@ +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. diff --git a/modules/repository-url/licenses/commons-codec-LICENSE.txt b/modules/repository-url/licenses/commons-codec-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/modules/repository-url/licenses/commons-codec-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. diff --git a/modules/repository-url/licenses/commons-codec-NOTICE.txt b/modules/repository-url/licenses/commons-codec-NOTICE.txt new file mode 100644 index 0000000000000..56916449bbe10 --- /dev/null +++ b/modules/repository-url/licenses/commons-codec-NOTICE.txt @@ -0,0 +1,17 @@ +Apache Commons Codec +Copyright 2002-2015 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java +contains test data from http://aspell.net/test/orig/batch0.tab. +Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + +=============================================================================== + +The content of package org.apache.commons.codec.language.bm has been translated +from the original php source code available at http://stevemorse.org/phoneticinfo.htm +with permission from the original authors. +Original source copyright: +Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/plugins/analysis-phonetic/licenses/commons-codec-LICENSE.txt b/plugins/analysis-phonetic/licenses/commons-codec-LICENSE.txt new file mode 100644 index 0000000000000..57bc88a15a0ee --- /dev/null +++ b/plugins/analysis-phonetic/licenses/commons-codec-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. + diff --git a/plugins/analysis-phonetic/licenses/commons-codec-NOTICE.txt b/plugins/analysis-phonetic/licenses/commons-codec-NOTICE.txt new file mode 100644 index 0000000000000..72eb32a902458 --- /dev/null +++ b/plugins/analysis-phonetic/licenses/commons-codec-NOTICE.txt @@ -0,0 +1,5 @@ +Apache Commons CLI +Copyright 2001-2009 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). diff --git a/plugins/discovery-azure-classic/licenses/commons-codec-LICENSE.txt b/plugins/discovery-azure-classic/licenses/commons-codec-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/discovery-azure-classic/licenses/commons-codec-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. diff --git a/plugins/discovery-azure-classic/licenses/commons-codec-NOTICE.txt b/plugins/discovery-azure-classic/licenses/commons-codec-NOTICE.txt new file mode 100644 index 0000000000000..56916449bbe10 --- /dev/null +++ b/plugins/discovery-azure-classic/licenses/commons-codec-NOTICE.txt @@ -0,0 +1,17 @@ +Apache Commons Codec +Copyright 2002-2015 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java +contains test data from http://aspell.net/test/orig/batch0.tab. +Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + +=============================================================================== + +The content of package org.apache.commons.codec.language.bm has been translated +from the original php source code available at http://stevemorse.org/phoneticinfo.htm +with permission from the original authors. +Original source copyright: +Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/plugins/discovery-ec2/licenses/commons-codec-LICENSE.txt b/plugins/discovery-ec2/licenses/commons-codec-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/discovery-ec2/licenses/commons-codec-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. diff --git a/plugins/discovery-ec2/licenses/commons-codec-NOTICE.txt b/plugins/discovery-ec2/licenses/commons-codec-NOTICE.txt new file mode 100644 index 0000000000000..56916449bbe10 --- /dev/null +++ b/plugins/discovery-ec2/licenses/commons-codec-NOTICE.txt @@ -0,0 +1,17 @@ +Apache Commons Codec +Copyright 2002-2015 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java +contains test data from http://aspell.net/test/orig/batch0.tab. +Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + +=============================================================================== + +The content of package org.apache.commons.codec.language.bm has been translated +from the original php source code available at http://stevemorse.org/phoneticinfo.htm +with permission from the original authors. +Original source copyright: +Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/plugins/discovery-ec2/licenses/jackson-LICENSE b/plugins/discovery-ec2/licenses/jackson-LICENSE new file mode 100644 index 0000000000000..f5f45d26a49d6 --- /dev/null +++ b/plugins/discovery-ec2/licenses/jackson-LICENSE @@ -0,0 +1,8 @@ +This copy of Jackson JSON processor streaming parser/generator is licensed under the +Apache (Software) License, version 2.0 ("the License"). +See the License for details about distribution rights, and the +specific rights regarding derivate works. + +You may obtain a copy of the License at: + +http://www.apache.org/licenses/LICENSE-2.0 diff --git a/plugins/discovery-ec2/licenses/jackson-NOTICE b/plugins/discovery-ec2/licenses/jackson-NOTICE new file mode 100644 index 0000000000000..4c976b7b4cc58 --- /dev/null +++ b/plugins/discovery-ec2/licenses/jackson-NOTICE @@ -0,0 +1,20 @@ +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. diff --git a/plugins/discovery-gce/licenses/commons-codec-LICENSE.txt b/plugins/discovery-gce/licenses/commons-codec-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/discovery-gce/licenses/commons-codec-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. diff --git a/plugins/discovery-gce/licenses/commons-codec-NOTICE.txt b/plugins/discovery-gce/licenses/commons-codec-NOTICE.txt new file mode 100644 index 0000000000000..56916449bbe10 --- /dev/null +++ b/plugins/discovery-gce/licenses/commons-codec-NOTICE.txt @@ -0,0 +1,17 @@ +Apache Commons Codec +Copyright 2002-2015 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java +contains test data from http://aspell.net/test/orig/batch0.tab. +Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + +=============================================================================== + +The content of package org.apache.commons.codec.language.bm has been translated +from the original php source code available at http://stevemorse.org/phoneticinfo.htm +with permission from the original authors. +Original source copyright: +Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/plugins/discovery-gce/licenses/jackson-LICENSE.txt b/plugins/discovery-gce/licenses/jackson-LICENSE.txt new file mode 100644 index 0000000000000..f5f45d26a49d6 --- /dev/null +++ b/plugins/discovery-gce/licenses/jackson-LICENSE.txt @@ -0,0 +1,8 @@ +This copy of Jackson JSON processor streaming parser/generator is licensed under the +Apache (Software) License, version 2.0 ("the License"). +See the License for details about distribution rights, and the +specific rights regarding derivate works. + +You may obtain a copy of the License at: + +http://www.apache.org/licenses/LICENSE-2.0 diff --git a/plugins/discovery-gce/licenses/jackson-NOTICE.txt b/plugins/discovery-gce/licenses/jackson-NOTICE.txt new file mode 100644 index 0000000000000..4c976b7b4cc58 --- /dev/null +++ b/plugins/discovery-gce/licenses/jackson-NOTICE.txt @@ -0,0 +1,20 @@ +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. diff --git a/plugins/repository-hdfs/licenses/commons-codec-LICENSE.txt b/plugins/repository-hdfs/licenses/commons-codec-LICENSE.txt new file mode 100644 index 0000000000000..75b52484ea471 --- /dev/null +++ b/plugins/repository-hdfs/licenses/commons-codec-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. diff --git a/plugins/repository-hdfs/licenses/commons-codec-NOTICE.txt b/plugins/repository-hdfs/licenses/commons-codec-NOTICE.txt new file mode 100644 index 0000000000000..efc098ca3ee67 --- /dev/null +++ b/plugins/repository-hdfs/licenses/commons-codec-NOTICE.txt @@ -0,0 +1,17 @@ +Apache Commons Codec +Copyright 2002-2014 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java +contains test data from http://aspell.net/test/orig/batch0.tab. +Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + +=============================================================================== + +The content of package org.apache.commons.codec.language.bm has been translated +from the original php source code available at http://stevemorse.org/phoneticinfo.htm +with permission from the original authors. +Original source copyright: +Copyright (c) 2008 Alexander Beider & Stephen P. Morse. diff --git a/plugins/repository-hdfs/licenses/slf4j-api-LICENSE.txt b/plugins/repository-hdfs/licenses/slf4j-api-LICENSE.txt new file mode 100644 index 0000000000000..8fda22f4d72f6 --- /dev/null +++ b/plugins/repository-hdfs/licenses/slf4j-api-LICENSE.txt @@ -0,0 +1,21 @@ +Copyright (c) 2004-2014 QOS.ch +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. diff --git a/plugins/repository-hdfs/licenses/slf4j-api-NOTICE.txt b/plugins/repository-hdfs/licenses/slf4j-api-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/arrow.bulk.json b/rest-api-spec/src/main/resources/rest-api-spec/api/arrow.bulk.json new file mode 100644 index 0000000000000..a13b59ff78cd9 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/arrow.bulk.json @@ -0,0 +1,98 @@ +{ + "bulk":{ + "documentation":{ + "url":"https://www.elastic.co/guide/en/elasticsearch/reference/master/docs-bulk.html", + "description":"Allows to perform multiple index/update/delete operations in a single request using the Arrow IPC streaming format." + }, + "stability":"experimental", + "visibility":"public", + "headers":{ + "accept": [ "application/json"], + "content_type": ["application/vnd.apache.arrow.stream"] + }, + "url":{ + "paths":[ + { + "path":"/_arrow/_bulk", + "methods":[ + "POST", + "PUT" + ] + }, + { + "path":"/_arrow/{index}/_bulk", + "methods":[ + "POST", + "PUT" + ], + "parts":{ + "index":{ + "type":"string", + "description":"Default index for items which don't provide one" + } + } + } + ] + }, + "params":{ + "wait_for_active_shards":{ + "type":"string", + "description":"Sets the number of shard copies that must be active before proceeding with the bulk operation. Defaults to 1, meaning the primary shard only. Set to `all` for all shard copies, otherwise set to any non-negative value less than or equal to the total number of copies for the shard (number of replicas + 1)" + }, + "refresh":{ + "type":"enum", + "options":[ + "true", + "false", + "wait_for" + ], + "description":"If `true` then refresh the affected shards to make this operation visible to search, if `wait_for` then wait for a refresh to make this operation visible to search, if `false` (the default) then do nothing with refreshes." + }, + "routing":{ + "type":"string", + "description":"Specific routing value" + }, + "timeout":{ + "type":"time", + "description":"Explicit operation timeout" + }, + "_source":{ + "type":"list", + "description":"True or false to return the _source field or not, or default list of fields to return, can be overridden on each sub-request" + }, + "_source_excludes":{ + "type":"list", + "description":"Default list of fields to exclude from the returned _source field, can be overridden on each sub-request" + }, + "_source_includes":{ + "type":"list", + "description":"Default list of fields to extract and return from the _source field, can be overridden on each sub-request" + }, + "pipeline":{ + "type":"string", + "description":"The pipeline id to preprocess incoming documents with" + }, + "require_alias": { + "type": "boolean", + "description": "If true, the request’s actions must target an index alias. Defaults to false." + }, + "require_data_stream": { + "type": "boolean", + "description": "If true, the request's actions must target a data stream (existing or to-be-created). Default to false" + }, + "list_executed_pipelines": { + "type": "boolean", + "description": "Sets list_executed_pipelines for all incoming documents. Defaults to unset (false)" + }, + "include_source_on_error": { + "type": "boolean", + "description": "True or false if to include the document source in the error message in case of parsing errors. Defaults to true." + } + }, + "body":{ + "description":"The operation definition and data (action-data pairs), separated by newlines", + "required":true, + "serialize":"bulk" + } + } +} diff --git a/server/build.gradle b/server/build.gradle index 70e49abb306bf..18202a25b0f3d 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -38,7 +38,6 @@ dependencies { api project(":libs:plugin-analysis-api") api project(':libs:grok') api project(":libs:tdigest") - implementation project(":libs:arrow") implementation project(":libs:simdvec") implementation project(":libs:entitlement") @@ -84,8 +83,6 @@ dependencies { internalClusterTestImplementation(project(':modules:reindex')) internalClusterTestImplementation(project(':modules:mapper-extras')) internalClusterTestImplementation(project(':modules:data-streams')) - - javaRestTestImplementation(project(":libs:arrow")) } spotless { diff --git a/server/src/javaRestTest/java/org/elasticsearch/action/bulk/arrow/BulkArrowIT.java b/server/src/javaRestTest/java/org/elasticsearch/action/bulk/arrow/BulkArrowIT.java deleted file mode 100644 index 0580ac66796e5..0000000000000 --- a/server/src/javaRestTest/java/org/elasticsearch/action/bulk/arrow/BulkArrowIT.java +++ /dev/null @@ -1,182 +0,0 @@ -/* - * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public - * License v3.0 only", or the "Server Side Public License, v 1". - */ - -package org.elasticsearch.action.bulk.arrow; - -import org.apache.arrow.vector.IntVector; -import org.apache.arrow.vector.VarCharVector; -import org.apache.arrow.vector.VectorSchemaRoot; -import org.apache.arrow.vector.ipc.ArrowStreamWriter; -import org.apache.arrow.vector.types.pojo.ArrowType; -import org.apache.arrow.vector.types.pojo.Field; -import org.apache.arrow.vector.types.pojo.FieldType; -import org.apache.arrow.vector.types.pojo.Schema; -import org.apache.arrow.vector.util.Text; -import org.apache.http.entity.ByteArrayEntity; -import org.apache.http.entity.ContentType; -import org.elasticsearch.arrow.Arrow; -import org.elasticsearch.client.Request; -import org.elasticsearch.test.cluster.ElasticsearchCluster; -import org.elasticsearch.test.cluster.local.distribution.DistributionType; -import org.elasticsearch.test.rest.ESRestTestCase; -import org.elasticsearch.xcontent.XContentParserConfiguration; -import org.elasticsearch.xcontent.XContentType; -import org.junit.ClassRule; - -import java.io.ByteArrayOutputStream; -import java.util.List; - -/** - * End-to-end test for Arrow bulk ingestion. Tests for the various Arrow datatypes and - * bulk actions are in {@code ArrowBulkIncrementalParserTests} - */ -public class BulkArrowIT extends ESRestTestCase { - - @ClassRule - public static ElasticsearchCluster cluster = ElasticsearchCluster.local() - .distribution(DistributionType.INTEG_TEST) - .setting("xpack.security.enabled", "false") - .setting("xpack.license.self_generated.type", "basic") - .build(); - - @Override - protected String getTestRestCluster() { - return cluster.getHttpAddresses(); - } - - public void testBulk() throws Exception { - - // Create a dataframe with two columns: integer and string - Field intField = new Field("ints", FieldType.nullable(new ArrowType.Int(32, true)), null); - Field strField = new Field("strings", FieldType.nullable(new ArrowType.Utf8()), null); - Schema schema = new Schema(List.of(intField, strField)); - - int batchCount = 7; - int rowCount = 11; - - byte[] payload; - - // Create vectors and write them to a byte array - try ( - var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); - var root = VectorSchemaRoot.create(schema, allocator); - ) { - var baos = new ByteArrayOutputStream(); - IntVector intVector = (IntVector) root.getVector(0); - VarCharVector stringVector = (VarCharVector) root.getVector(1); - - try (ArrowStreamWriter writer = new ArrowStreamWriter(root, null, baos)) { - for (int batch = 0; batch < batchCount; batch++) { - intVector.allocateNew(rowCount); - stringVector.allocateNew(rowCount); - for (int row = 0; row < rowCount; row++) { - int globalRow = row + batch * rowCount; - intVector.set(row, globalRow); - stringVector.set(row, new Text("row" + globalRow)); - } - root.setRowCount(rowCount); - writer.writeBatch(); - } - } - payload = baos.toByteArray(); - } - - { - // Bulk insert the arrow stream - var request = new Request("POST", "/arrow_bulk_test/_bulk"); - request.addParameter("refresh", "wait_for"); - request.setEntity(new ByteArrayEntity(payload, ContentType.create(Arrow.MEDIA_TYPE))); - - var response = client().performRequest(request); - var result = XContentType.JSON.xContent() - .createParser(XContentParserConfiguration.EMPTY, response.getEntity().getContent()) - .map(); - - assertEquals(Boolean.FALSE, result.get("errors")); - assertEquals(batchCount * rowCount, ((List) result.get("items")).size()); - } - - { - // Check that the index effectively contains what we sent - var request = new Request("GET", "/arrow_bulk_test/_count"); - var response = client().performRequest(request); - var result = XContentType.JSON.xContent() - .createParser(XContentParserConfiguration.EMPTY, response.getEntity().getContent()) - .map(); - - assertEquals(batchCount * rowCount, result.get("count")); - } - } - - public void testDictionary() throws Exception { - - // DictionaryEncoding encoding = new DictionaryEncoding() - - // Create a dataframe with two columns: integer and string - Field intField = new Field("ints", FieldType.nullable(new ArrowType.Int(32, true)), null); - Field strField = new Field("strings", FieldType.nullable(new ArrowType.Utf8()), null); - Schema schema = new Schema(List.of(intField, strField)); - - int batchCount = 7; - int rowCount = 11; - - byte[] payload; - - // Create vectors and write them to a byte array - try ( - var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); - var root = VectorSchemaRoot.create(schema, allocator); - ) { - var baos = new ByteArrayOutputStream(); - IntVector intVector = (IntVector) root.getVector(0); - VarCharVector stringVector = (VarCharVector) root.getVector(1); - - try (ArrowStreamWriter writer = new ArrowStreamWriter(root, null, baos)) { - for (int batch = 0; batch < batchCount; batch++) { - intVector.allocateNew(rowCount); - stringVector.allocateNew(rowCount); - for (int row = 0; row < rowCount; row++) { - int globalRow = row + batch * rowCount; - intVector.set(row, globalRow); - stringVector.set(row, new Text("row" + globalRow)); - } - root.setRowCount(rowCount); - writer.writeBatch(); - } - } - payload = baos.toByteArray(); - } - - { - // Bulk insert the arrow stream - var request = new Request("POST", "/arrow_bulk_test/_bulk"); - request.addParameter("refresh", "wait_for"); - request.setEntity(new ByteArrayEntity(payload, ContentType.create(Arrow.MEDIA_TYPE))); - - var response = client().performRequest(request); - var result = XContentType.JSON.xContent() - .createParser(XContentParserConfiguration.EMPTY, response.getEntity().getContent()) - .map(); - - assertEquals(Boolean.FALSE, result.get("errors")); - assertEquals(batchCount * rowCount, ((List) result.get("items")).size()); - } - - { - // Check that the index effectively contains what we sent - var request = new Request("GET", "/arrow_bulk_test/_count"); - var response = client().performRequest(request); - var result = XContentType.JSON.xContent() - .createParser(XContentParserConfiguration.EMPTY, response.getEntity().getContent()) - .map(); - - assertEquals(batchCount * rowCount, result.get("count")); - } - } -} diff --git a/server/src/main/java/module-info.java b/server/src/main/java/module-info.java index 1fac6d811a47d..86de9b9d8bc13 100644 --- a/server/src/main/java/module-info.java +++ b/server/src/main/java/module-info.java @@ -16,13 +16,17 @@ requires java.security.jgss; requires java.sql; requires java.management; + requires jdk.unsupported; requires java.net.http; // required by ingest-geoip's dependency maxmind.geoip2 https://github.com/elastic/elasticsearch/issues/93553 requires org.elasticsearch.cli; + requires org.elasticsearch.base; requires org.elasticsearch.nativeaccess; requires org.elasticsearch.geo; requires org.elasticsearch.lz4; requires org.elasticsearch.securesm; + requires org.elasticsearch.xcontent; + requires org.elasticsearch.logging; requires org.elasticsearch.plugin; requires org.elasticsearch.plugin.analysis; requires org.elasticsearch.grok; @@ -51,10 +55,6 @@ requires org.apache.lucene.queryparser; requires org.apache.lucene.sandbox; requires org.apache.lucene.suggest; - requires org.elasticsearch.arrow; - requires org.elasticsearch.xcontent; - requires org.elasticsearch.base; - requires org.elasticsearch.logging; exports org.elasticsearch; exports org.elasticsearch.action; diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java index ed4e0e2de051f..3c81e3a94e03f 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java @@ -49,25 +49,25 @@ public final class BulkRequestParser extends AbstractBulkRequestParser { // Remove deprecation logger when its usages in checkBulkActionIsProperlyClosed are removed private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(BulkRequestParser.class); - public static final Set SUPPORTED_ACTIONS = Set.of("create", "index", "update", "delete"); + private static final Set SUPPORTED_ACTIONS = Set.of("create", "index", "update", "delete"); private static final String STRICT_ACTION_PARSING_WARNING_KEY = "bulk_request_strict_action_parsing"; - public static final ParseField INDEX = new ParseField("_index"); - public static final ParseField TYPE = new ParseField("_type"); - public static final ParseField ID = new ParseField("_id"); - public static final ParseField ROUTING = new ParseField("routing"); - public static final ParseField OP_TYPE = new ParseField("op_type"); - public static final ParseField VERSION = new ParseField("version"); - public static final ParseField VERSION_TYPE = new ParseField("version_type"); - public static final ParseField RETRY_ON_CONFLICT = new ParseField("retry_on_conflict"); - public static final ParseField PIPELINE = new ParseField("pipeline"); - public static final ParseField SOURCE = new ParseField("_source"); - public static final ParseField IF_SEQ_NO = new ParseField("if_seq_no"); - public static final ParseField IF_PRIMARY_TERM = new ParseField("if_primary_term"); - public static final ParseField REQUIRE_ALIAS = new ParseField(DocWriteRequest.REQUIRE_ALIAS); - public static final ParseField REQUIRE_DATA_STREAM = new ParseField(DocWriteRequest.REQUIRE_DATA_STREAM); - public static final ParseField LIST_EXECUTED_PIPELINES = new ParseField(DocWriteRequest.LIST_EXECUTED_PIPELINES); - public static final ParseField DYNAMIC_TEMPLATES = new ParseField("dynamic_templates"); + private static final ParseField INDEX = new ParseField("_index"); + private static final ParseField TYPE = new ParseField("_type"); + private static final ParseField ID = new ParseField("_id"); + private static final ParseField ROUTING = new ParseField("routing"); + private static final ParseField OP_TYPE = new ParseField("op_type"); + private static final ParseField VERSION = new ParseField("version"); + private static final ParseField VERSION_TYPE = new ParseField("version_type"); + private static final ParseField RETRY_ON_CONFLICT = new ParseField("retry_on_conflict"); + private static final ParseField PIPELINE = new ParseField("pipeline"); + private static final ParseField SOURCE = new ParseField("_source"); + private static final ParseField IF_SEQ_NO = new ParseField("if_seq_no"); + private static final ParseField IF_PRIMARY_TERM = new ParseField("if_primary_term"); + private static final ParseField REQUIRE_ALIAS = new ParseField(DocWriteRequest.REQUIRE_ALIAS); + private static final ParseField REQUIRE_DATA_STREAM = new ParseField(DocWriteRequest.REQUIRE_DATA_STREAM); + private static final ParseField LIST_EXECUTED_PIPELINES = new ParseField(DocWriteRequest.LIST_EXECUTED_PIPELINES); + private static final ParseField DYNAMIC_TEMPLATES = new ParseField("dynamic_templates"); // TODO: Remove this parameter once the BulkMonitoring endpoint has been removed // for CompatibleApi V7 this means to deprecate on type, for V8+ it means to throw an error diff --git a/server/src/main/java/org/elasticsearch/rest/RestController.java b/server/src/main/java/org/elasticsearch/rest/RestController.java index 647fc2ade4596..5037a4f294638 100644 --- a/server/src/main/java/org/elasticsearch/rest/RestController.java +++ b/server/src/main/java/org/elasticsearch/rest/RestController.java @@ -86,6 +86,7 @@ public class RestController implements HttpServerTransport.Dispatcher { * https://fetch.spec.whatwg.org/#cors-safelisted-request-header */ static final Set SAFELISTED_MEDIA_TYPES = Set.of("application/x-www-form-urlencoded", "multipart/form-data", "text/plain"); + static final String ARROW_STREAM = "application/vnd.apache.arrow.stream"; static final String ELASTIC_PRODUCT_HTTP_HEADER = "X-elastic-product"; static final String ELASTIC_PRODUCT_HTTP_HEADER_VALUE = "Elasticsearch"; @@ -403,9 +404,9 @@ private void dispatchRequest( final XContentType xContentType = request.getXContentType(); // TODO consider refactoring to handler.supportsContentStream(xContentType). It is only used with JSON and SMILE if (handler.supportsBulkContent() - && xContentType != null // null for Arrow format && XContentType.JSON != xContentType.canonical() - && XContentType.SMILE != xContentType.canonical()) { + && XContentType.SMILE != xContentType.canonical() + && (request.getParsedContentType().mediaTypeWithoutParameters().equals(ARROW_STREAM) == false)) { channel.sendResponse( RestResponse.createSimpleErrorResponse( channel, diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java index dbb6156d2b342..4d90d3c375cc7 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java @@ -17,7 +17,6 @@ import org.elasticsearch.action.bulk.BulkRequestParser; import org.elasticsearch.action.bulk.BulkShardRequest; import org.elasticsearch.action.bulk.IncrementalBulkService; -import org.elasticsearch.action.bulk.arrow.ArrowBulkRequestParser; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.cluster.metadata.DataStream; @@ -84,11 +83,6 @@ public List routes() { ); } - @Override - public boolean mediaTypesValid(RestRequest request) { - return super.mediaTypesValid(request) || ArrowBulkRequestParser.isArrowRequest(request); - } - @Override public String getName() { return "bulk_action"; @@ -152,7 +146,7 @@ private static Exception parseFailureException(Exception e) { } } - static class ChunkHandler implements BaseRestHandler.RequestBodyChunkConsumer { + public static class ChunkHandler implements BaseRestHandler.RequestBodyChunkConsumer { private final RestRequest request; @@ -165,15 +159,16 @@ static class ChunkHandler implements BaseRestHandler.RequestBodyChunkConsumer { private final ArrayDeque unParsedChunks = new ArrayDeque<>(4); private final ArrayList> items = new ArrayList<>(4); - ChunkHandler(boolean allowExplicitIndex, RestRequest request, Supplier handlerSupplier) { + public ChunkHandler(boolean allowExplicitIndex, RestRequest request, Supplier handlerSupplier) { + this(allowExplicitIndex, request, handlerSupplier, + new BulkRequestParser(true, RestUtils.getIncludeSourceOnError(request), request.getRestApiVersion()) + ); + } + + public ChunkHandler(boolean allowExplicitIndex, RestRequest request, Supplier handlerSupplier, + AbstractBulkRequestParser requestParser) { this.request = request; this.handlerSupplier = handlerSupplier; - AbstractBulkRequestParser requestParser; - if (ArrowBulkRequestParser.isArrowRequest(request)) { - requestParser = new ArrowBulkRequestParser(request); - } else { - requestParser = new BulkRequestParser(true, RestUtils.getIncludeSourceOnError(request), request.getRestApiVersion()); - } this.parser = requestParser.incrementalParser( request.param("index"), request.param("routing"), diff --git a/server/src/main/resources/org/elasticsearch/bootstrap/security.policy b/server/src/main/resources/org/elasticsearch/bootstrap/security.policy index bcdbd450d420a..39be2ca49eab5 100644 --- a/server/src/main/resources/org/elasticsearch/bootstrap/security.policy +++ b/server/src/main/resources/org/elasticsearch/bootstrap/security.policy @@ -73,11 +73,6 @@ grant codeBase "${codebase.elasticsearch-simdvec}" { permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; }; -grant codebase "${codebase.jackson-databind}" { - permission java.lang.RuntimePermission "accessDeclaredMembers"; - permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; -}; - // Arrow uses Unsafe to access memory and accesses a private field in java.nio.Buffer // See also additional global permissions below grant codebase "${codebase.arrow-memory-core}" { diff --git a/server/src/test/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParserTests.java b/server/src/test/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParserTests.java deleted file mode 100644 index c29b34d7e00ae..0000000000000 --- a/server/src/test/java/org/elasticsearch/action/bulk/arrow/ArrowBulkIncrementalParserTests.java +++ /dev/null @@ -1,473 +0,0 @@ -/* - * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public - * License v3.0 only", or the "Server Side Public License, v 1". - */ - -package org.elasticsearch.action.bulk.arrow; - -import org.apache.arrow.vector.FieldVector; -import org.apache.arrow.vector.IntVector; -import org.apache.arrow.vector.VarCharVector; -import org.apache.arrow.vector.VectorSchemaRoot; -import org.apache.arrow.vector.complex.MapVector; -import org.apache.arrow.vector.complex.StructVector; -import org.apache.arrow.vector.dictionary.Dictionary; -import org.apache.arrow.vector.dictionary.DictionaryEncoder; -import org.apache.arrow.vector.dictionary.DictionaryProvider; -import org.apache.arrow.vector.ipc.ArrowStreamWriter; -import org.apache.arrow.vector.types.Types; -import org.apache.arrow.vector.types.pojo.ArrowType; -import org.apache.arrow.vector.types.pojo.DictionaryEncoding; -import org.apache.arrow.vector.types.pojo.Field; -import org.apache.arrow.vector.types.pojo.FieldType; -import org.apache.arrow.vector.types.pojo.Schema; -import org.apache.arrow.vector.util.Text; -import org.elasticsearch.action.DocWriteRequest; -import org.elasticsearch.action.delete.DeleteRequest; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.update.UpdateRequest; -import org.elasticsearch.arrow.Arrow; -import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.core.RestApiVersion; -import org.elasticsearch.search.fetch.subphase.FetchSourceContext; -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xcontent.XContentParserConfiguration; -import org.elasticsearch.xcontent.XContentType; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.function.BiConsumer; -import java.util.function.Consumer; - -import static org.hamcrest.Matchers.equalTo; - -public class ArrowBulkIncrementalParserTests extends ESTestCase { - - // ----- Test Arrow batches and incremental parsing - - public void testBatchingAndChunking() throws IOException { - checkBatchingAndChunking(1, 10, false); - checkBatchingAndChunking(1, 10, true); - checkBatchingAndChunking(2, 10, false); - checkBatchingAndChunking(2, 10, true); - } - - /** Create a payload for a 1-column dataframe (int and string), given a number of batches and rows per batch */ - private void checkBatchingAndChunking(int batchCount, int rowCount, boolean incremental) throws IOException { - byte[] payload; - - // Create a dataframe with two columns: integer and string - Field intField = new Field("ints", FieldType.nullable(new ArrowType.Int(32, true)), null); - Field strField = new Field("strings", FieldType.nullable(new ArrowType.Utf8()), null); - Schema schema = new Schema(List.of(intField, strField)); - - // Create vectors and write them to a byte array - try ( - var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); - var root = VectorSchemaRoot.create(schema, allocator); - ) { - var baos = new ByteArrayOutputStream(); - IntVector intVector = (IntVector) root.getVector(0); - VarCharVector stringVector = (VarCharVector) root.getVector(1); - - try (ArrowStreamWriter writer = new ArrowStreamWriter(root, null, baos)) { - for (int batch = 0; batch < batchCount; batch++) { - intVector.allocateNew(rowCount); - stringVector.allocateNew(rowCount); - for (int row = 0; row < rowCount; row++) { - int globalRow = row + batch * rowCount; - intVector.set(row, globalRow); - stringVector.set(row, new Text("row" + globalRow)); - } - root.setRowCount(rowCount); - writer.writeBatch(); - } - } - payload = baos.toByteArray(); - } - - var operations = new ArrayList>(); - try (var parser = createParser("test", operations)) { - parse(parser, payload, incremental); - } - ; - - assertEquals(batchCount * rowCount, operations.size()); - - for (int i = 0; i < operations.size(); i++) { - IndexRequest operation = (IndexRequest) operations.get(i); - - assertEquals(DocWriteRequest.OpType.INDEX, operation.opType()); - assertEquals("test", operation.index()); - - assertEquals(XContentType.CBOR, operation.getContentType()); - - var map = operation.sourceAsMap(); - assertEquals(i, map.get("ints")); - assertEquals("row" + i, map.get("strings")); - } - } - - public void testInlineIdAndIndex() throws Exception { - byte[] payload; - - Field indexField = new Field("_index", FieldType.nullable(new ArrowType.Utf8()), null); - Field idField = new Field("_id", FieldType.nullable(new ArrowType.Utf8()), null); - Field intField = new Field("ints", FieldType.nullable(new ArrowType.Int(32, true)), null); - Field strField = new Field("strings", FieldType.nullable(new ArrowType.Utf8()), null); - Schema schema = new Schema(List.of(indexField, idField, intField, strField)); - - try ( - var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); - var root = VectorSchemaRoot.create(schema, allocator); - ) { - var baos = new ByteArrayOutputStream(); - VarCharVector indexVector = (VarCharVector) root.getVector(0); - VarCharVector idVector = (VarCharVector) root.getVector(1); - IntVector intVector = (IntVector) root.getVector(2); - VarCharVector stringVector = (VarCharVector) root.getVector(3); - - try (ArrowStreamWriter writer = new ArrowStreamWriter(root, null, baos)) { - indexVector.allocateNew(4); - idVector.allocateNew(4); - intVector.allocateNew(4); - stringVector.allocateNew(4); - - // No index, no id - indexVector.setNull(0); - idVector.setNull(0); - stringVector.set(0, new Text("row0")); - intVector.set(0, 0); - - // No index, id - indexVector.setNull(1); - idVector.set(1, new Text("id1")); - stringVector.set(1, new Text("row1")); - intVector.set(1, 1); - - // Index, no id - indexVector.set(2, new Text("index2")); - idVector.setNull(2); - stringVector.set(2, new Text("row2")); - intVector.set(2, 2); - - // Index & id - indexVector.set(3, new Text("index3")); - idVector.set(3, new Text("id3")); - stringVector.set(1, new Text("row3")); - intVector.set(1, 3); - - root.setRowCount(4); - writer.writeBatch(); - } - payload = baos.toByteArray(); - } - - var operations = new ArrayList>(); - try (var parser = createParser("defaultIndex", operations)) { - parse(parser, payload, false); - } - ; - - IndexRequest operation = (IndexRequest) operations.get(0); - assertEquals("defaultIndex", operation.index()); - assertEquals(null, operation.id()); - - operation = (IndexRequest) operations.get(1); - assertEquals("defaultIndex", operation.index()); - assertEquals("id1", operation.id()); - - operation = (IndexRequest) operations.get(2); - assertEquals("index2", operation.index()); - assertEquals(null, operation.id()); - - operation = (IndexRequest) operations.get(3); - assertEquals("index3", operation.index()); - assertEquals("id3", operation.id()); - - } - - // ----- Test action decoding - - /** Action as a map of (string, string) */ - public void testActionsAsStringMap() throws Exception { - - try ( - var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); - var vector = new MapVector("action", allocator, FieldType.nullable(new ArrowType.Map(false)), null); - var parser = createParser("default-index", List.of()) - ) { - var w = vector.getWriter(); - - w.startMap(); - - // Override operation type (default is create) - w.startEntry(); - w.key().varChar().writeVarChar("op_type"); - w.value().varChar().writeVarChar("update"); - w.endEntry(); - - // Override default "default-index" index - w.startEntry(); - w.key().varChar().writeVarChar("_index"); - w.value().varChar().writeVarChar("first-index"); - w.endEntry(); - - // Set if_seq_no as a string, to test a lazy approach with a simple (string, string) map - w.startEntry(); - w.key().varChar().writeVarChar("if_seq_no"); - w.value().varChar().writeVarChar("3"); - w.endEntry(); - - w.endMap(); - - w.startMap(); - - // Override default "default-index" index - w.startEntry(); - w.key().varChar().writeVarChar("_index"); - w.value().varChar().writeVarChar("second-index"); - w.endEntry(); - - // Override operation type (default is create) - w.startEntry(); - w.key().varChar().writeVarChar("op_type"); - w.value().varChar().writeVarChar("index"); - w.endEntry(); - - // Set version as a string, to test a lazy approach with a simple (string, string) map - w.startEntry(); - w.key().varChar().writeVarChar("if_seq_no"); - w.value().varChar().writeVarChar("4"); - w.endEntry(); - - w.endMap(); - - vector.setValueCount(w.getPosition()); - // Value type is varchar - assertEquals(Types.MinorType.VARCHAR, vector.getChildrenFromFields().get(0).getChildrenFromFields().get(1).getMinorType()); - - { - var request = parser.parseAction(vector, 0, null, null); - assertEquals(DocWriteRequest.OpType.UPDATE, request.opType()); - assertEquals("first-index", request.index()); - assertEquals(3, request.ifSeqNo()); - } - - { - var request = parser.parseAction(vector, 1, null, null); - assertEquals(DocWriteRequest.OpType.INDEX, request.opType()); - assertEquals("second-index", request.index()); - assertEquals(4, request.ifSeqNo()); - } - } - } - - /** Action as a map of (string, union(string, int)) */ - public void testActionsAsUnionMap() throws Exception { - - try ( - var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); - var vector = new MapVector("action", allocator, FieldType.nullable(new ArrowType.Map(false)), null); - var parser = createParser("default-index", List.of()) - ) { - var w = vector.getWriter(); - - w.startMap(); - - // Override operation type (default is create) - w.startEntry(); - w.key().varChar().writeVarChar("op_type"); - w.value().varChar().writeVarChar("update"); - w.endEntry(); - - // Override default "default-index" index - w.startEntry(); - w.key().varChar().writeVarChar("_index"); - w.value().varChar().writeVarChar("some-index"); - w.endEntry(); - - // Set version as a number. This promotes the value field to a union type - w.startEntry(); - w.key().varChar().writeVarChar("if_seq_no"); - w.value().integer().writeInt(3); - w.endEntry(); - - w.endMap(); - - vector.setValueCount(w.getPosition()); - var request = parser.parseAction(vector, 0, null, null); - - // Value type is a union - assertEquals(Types.MinorType.UNION, vector.getChildrenFromFields().get(0).getChildrenFromFields().get(1).getMinorType()); - - assertEquals(DocWriteRequest.OpType.UPDATE, request.opType()); - assertEquals("some-index", request.index()); - assertEquals(3, request.ifSeqNo()); - } - } - - /** Action as a struct */ - public void testActionsAsStruct() throws Exception { - - try ( - var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); - var vector = new StructVector("action", allocator, FieldType.nullable(new ArrowType.Struct()), null); - var parser = createParser("default-index", List.of()) - ) { - var w = vector.getWriter(); - - w.start(); - w.varChar("op_type").writeVarChar("update"); - w.varChar("_index").writeVarChar("first-index"); - w.integer("if_seq_no").writeInt(3); - w.end(); - - w.start(); - w.varChar("op_type").writeVarChar("index"); - w.varChar("_index").writeVarChar("second-index"); - w.integer("if_seq_no").writeInt(4); - w.end(); - - vector.setValueCount(w.getPosition()); - - { - var request = parser.parseAction(vector, 0, null, null); - assertEquals(DocWriteRequest.OpType.UPDATE, request.opType()); - assertEquals("first-index", request.index()); - assertEquals(3, request.ifSeqNo()); - } - - { - var request = parser.parseAction(vector, 1, null, null); - assertEquals(DocWriteRequest.OpType.INDEX, request.opType()); - assertEquals("second-index", request.index()); - assertEquals(4, request.ifSeqNo()); - } - } - } - - // ----- Dictionary encoding - public void testDictionaryEncoding() throws Exception { - - ByteArrayOutputStream payload = new ByteArrayOutputStream(); - - try ( - var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); - VarCharVector dictVector = new VarCharVector("dict", allocator); - VarCharVector vector = new VarCharVector("data_field", allocator); - DictionaryProvider.MapDictionaryProvider dictionaryProvider = new DictionaryProvider.MapDictionaryProvider(); - ) { - // create dictionary lookup vector - dictVector.allocateNewSafe(); - dictVector.setSafe(0, "aa".getBytes()); - dictVector.setSafe(1, "bb".getBytes()); - dictVector.setSafe(2, "cc".getBytes()); - dictVector.setValueCount(3); - - // create dictionary - long dictionaryId = 1L; - Dictionary dictionary = new Dictionary(dictVector, new DictionaryEncoding(dictionaryId, false, /*indexType=*/null)); - - dictionaryProvider.put(dictionary); - - // create original data vector - vector.allocateNewSafe(); - vector.setSafe(0, "bb".getBytes()); - vector.setSafe(1, "bb".getBytes()); - vector.setSafe(2, "cc".getBytes()); - vector.setSafe(3, "aa".getBytes()); - vector.setValueCount(4); - - // Encode the vector with the dictionary - IntVector encodedVector = (IntVector) DictionaryEncoder.encode(vector, dictionary); - - // create VectorSchemaRoot - List fields = List.of(encodedVector.getField()); - List vectors = List.of(encodedVector); - - try ( - VectorSchemaRoot root = new VectorSchemaRoot(fields, vectors); - ArrowStreamWriter writer = new ArrowStreamWriter(root, dictionaryProvider, payload); - ) { - // write data - writer.start(); - writer.writeBatch(); - writer.end(); - } - - var operations = new ArrayList>(); - try (var parser = createParser("defaultIndex", operations)) { - parse(parser, payload.toByteArray(), false); - } - ; - - // Check that dictionary-encoded values were correctly decoded - assertEquals("bb", ((IndexRequest) operations.get(0)).sourceAsMap().get("data_field")); - assertEquals("bb", ((IndexRequest) operations.get(1)).sourceAsMap().get("data_field")); - assertEquals("cc", ((IndexRequest) operations.get(2)).sourceAsMap().get("data_field")); - assertEquals("aa", ((IndexRequest) operations.get(3)).sourceAsMap().get("data_field")); - } - } - - // ----- Utilities - - private static ArrowBulkIncrementalParser createParser(String defaultIndex, List> requests) { - - DocWriteRequest.OpType defaultOpType = DocWriteRequest.OpType.INDEX; - String defaultRouting = null; - FetchSourceContext defaultFetchSourceContext = null; - String defaultPipeline = null; - Boolean defaultRequireAlias = false; - Boolean defaultRequireDataStream = false; - Boolean defaultListExecutedPipelines = false; - - boolean allowExplicitIndex = true; - XContentType xContentType = null; - BiConsumer indexRequestConsumer = (r, t) -> requests.add(r); - Consumer updateRequestConsumer = requests::add; - Consumer deleteRequestConsumer = requests::add; - - return new ArrowBulkIncrementalParser( - defaultOpType, - defaultIndex, - defaultRouting, - defaultFetchSourceContext, - defaultPipeline, - defaultRequireAlias, - defaultRequireDataStream, - defaultListExecutedPipelines, - allowExplicitIndex, - xContentType, - XContentParserConfiguration.EMPTY.withRestApiVersion(RestApiVersion.current()), - indexRequestConsumer, - updateRequestConsumer, - deleteRequestConsumer - ); - } - - private void parse(ArrowBulkIncrementalParser parser, byte[] payload, boolean incremental) throws IOException { - - int consumed = 0; - var request = new BytesArray(payload); - - if (incremental) { - // Borrowed from BulkRequestParserTests - for (int i = 0; i < request.length() - 1; ++i) { - consumed += parser.parse(request.slice(consumed, i - consumed + 1), false); - } - consumed += parser.parse(request.slice(consumed, request.length() - consumed), true); - assertThat(consumed, equalTo(request.length())); - } else { - consumed = parser.parse(request, true); - } - - assertEquals(payload.length, consumed); - } -} diff --git a/server/src/test/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannelTests.java b/server/src/test/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannelTests.java deleted file mode 100644 index 1771c12e3bea0..0000000000000 --- a/server/src/test/java/org/elasticsearch/action/bulk/arrow/BytesReferenceChannelTests.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public - * License v3.0 only", or the "Server Side Public License, v 1". - */ - -package org.elasticsearch.action.bulk.arrow; - -import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.bytes.CompositeBytesReference; -import org.elasticsearch.test.ESTestCase; - -import java.io.IOException; -import java.nio.channels.Channels; -import java.util.ArrayList; - -public class BytesReferenceChannelTests extends ESTestCase { - - /** - * Check iteration on the buffers of a composite byteref - */ - public void testMultipleBuffers() throws IOException { - - var chunks = new ArrayList(); - byte[] chunk = null; - for (int i = 0; i < 0x100; i++) { - if (i % 0x10 == 0) { - chunk = new byte[0x10]; - chunks.add(new BytesArray(chunk)); - } - chunk[i % 0x10] = (byte) i; - } - - var bytesref = CompositeBytesReference.of(chunks.toArray(new BytesReference[0])); - - try (var channel = new BytesReferenceChannel(bytesref)) { - var in = Channels.newInputStream(channel); - - for (int i = 0; i < 0x100; i++) { - assertEquals(i, in.read()); - } - - assertEquals(-1, in.read()); - } - } -} diff --git a/x-pack/plugin/ent-search/licenses/jackson-LICENSE b/x-pack/plugin/ent-search/licenses/jackson-LICENSE new file mode 100644 index 0000000000000..f5f45d26a49d6 --- /dev/null +++ b/x-pack/plugin/ent-search/licenses/jackson-LICENSE @@ -0,0 +1,8 @@ +This copy of Jackson JSON processor streaming parser/generator is licensed under the +Apache (Software) License, version 2.0 ("the License"). +See the License for details about distribution rights, and the +specific rights regarding derivate works. + +You may obtain a copy of the License at: + +http://www.apache.org/licenses/LICENSE-2.0 diff --git a/x-pack/plugin/ent-search/licenses/jackson-NOTICE b/x-pack/plugin/ent-search/licenses/jackson-NOTICE new file mode 100644 index 0000000000000..4c976b7b4cc58 --- /dev/null +++ b/x-pack/plugin/ent-search/licenses/jackson-NOTICE @@ -0,0 +1,20 @@ +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. diff --git a/x-pack/plugin/ent-search/licenses/slf4j-api-LICENSE.txt b/x-pack/plugin/ent-search/licenses/slf4j-api-LICENSE.txt new file mode 100644 index 0000000000000..52055e61de46f --- /dev/null +++ b/x-pack/plugin/ent-search/licenses/slf4j-api-LICENSE.txt @@ -0,0 +1,21 @@ +Copyright (c) 2004-2014 QOS.ch +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. diff --git a/x-pack/plugin/ent-search/licenses/slf4j-api-NOTICE.txt b/x-pack/plugin/ent-search/licenses/slf4j-api-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/x-pack/plugin/esql/arrow/build.gradle b/x-pack/plugin/esql/arrow/build.gradle index fface6484e125..9f96e26cfbd98 100644 --- a/x-pack/plugin/esql/arrow/build.gradle +++ b/x-pack/plugin/esql/arrow/build.gradle @@ -12,11 +12,7 @@ dependencies { compileOnly project(':x-pack:plugin:esql:compute') compileOnly project(':x-pack:plugin:esql-core') compileOnly project(':x-pack:plugin:mapper-version') + implementation(project(":libs:arrow")) - implementation project(":libs:arrow") testImplementation project(':test:framework') } - -tasks.named("test").configure { - jvmArgs('--add-opens=java.base/java.nio=ALL-UNNAMED') -} diff --git a/x-pack/plugin/esql/qa/server/single-node/build.gradle b/x-pack/plugin/esql/qa/server/single-node/build.gradle index cea6ce20a2069..8554a77c36917 100644 --- a/x-pack/plugin/esql/qa/server/single-node/build.gradle +++ b/x-pack/plugin/esql/qa/server/single-node/build.gradle @@ -7,19 +7,9 @@ dependencies { javaRestTestImplementation project(xpackModule('esql:qa:testFixtures')) javaRestTestImplementation project(xpackModule('esql:qa:server')) javaRestTestImplementation project(xpackModule('esql:tools')) + javaRestTestImplementation project(":libs:arrow") yamlRestTestImplementation project(xpackModule('esql:qa:server')) - javaRestTestImplementation(project(":libs:arrow")) - javaRestTestImplementation("org.checkerframework:checker-qual:3.48.1") - javaRestTestImplementation("com.google.flatbuffers:flatbuffers-java:24.3.25") - javaRestTestImplementation("com.fasterxml.jackson.core:jackson-core:${versions.jackson}") - javaRestTestImplementation("com.fasterxml.jackson.core:jackson-annotations:${versions.jackson}") - javaRestTestImplementation("com.fasterxml.jackson.core:jackson-databind:${versions.jackson}") - javaRestTestImplementation("com.fasterxml.jackson.datatype:jackson-datatype-jsr310:${versions.jackson}") - javaRestTestImplementation("commons-codec:commons-codec:${versions.commonscodec}") - javaRestTestImplementation("org.slf4j:slf4j-api:${versions.slf4j}") - javaRestTestImplementation("org.slf4j:slf4j-nop:${versions.slf4j}") - clusterPlugins project(':plugins:mapper-size') clusterPlugins project(':plugins:mapper-murmur3') clusterPlugins project(':x-pack:plugin:inference:qa:test-service-plugin') @@ -37,7 +27,6 @@ restResources { tasks.named('javaRestTest') { usesDefaultDistribution("to be triaged") maxParallelForks = 1 - jvmArgs('--add-opens=java.base/java.nio=ALL-UNNAMED') } tasks.named('yamlRestTest') { diff --git a/x-pack/plugin/identity-provider/licenses/slf4j-api-LICENSE.txt b/x-pack/plugin/identity-provider/licenses/slf4j-api-LICENSE.txt new file mode 100644 index 0000000000000..c8e72c2cd4738 --- /dev/null +++ b/x-pack/plugin/identity-provider/licenses/slf4j-api-LICENSE.txt @@ -0,0 +1,21 @@ + Copyright (c) 2004-2017 QOS.ch + 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. diff --git a/x-pack/plugin/identity-provider/licenses/slf4j-api-NOTICE.txt b/x-pack/plugin/identity-provider/licenses/slf4j-api-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/x-pack/plugin/inference/licenses/jackson-LICENSE.txt b/x-pack/plugin/inference/licenses/jackson-LICENSE.txt new file mode 100644 index 0000000000000..f5f45d26a49d6 --- /dev/null +++ b/x-pack/plugin/inference/licenses/jackson-LICENSE.txt @@ -0,0 +1,8 @@ +This copy of Jackson JSON processor streaming parser/generator is licensed under the +Apache (Software) License, version 2.0 ("the License"). +See the License for details about distribution rights, and the +specific rights regarding derivate works. + +You may obtain a copy of the License at: + +http://www.apache.org/licenses/LICENSE-2.0 diff --git a/x-pack/plugin/inference/licenses/jackson-NOTICE.txt b/x-pack/plugin/inference/licenses/jackson-NOTICE.txt new file mode 100644 index 0000000000000..4c976b7b4cc58 --- /dev/null +++ b/x-pack/plugin/inference/licenses/jackson-NOTICE.txt @@ -0,0 +1,20 @@ +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. diff --git a/x-pack/plugin/security/licenses/slf4j-api-LICENSE.txt b/x-pack/plugin/security/licenses/slf4j-api-LICENSE.txt new file mode 100644 index 0000000000000..c8e72c2cd4738 --- /dev/null +++ b/x-pack/plugin/security/licenses/slf4j-api-LICENSE.txt @@ -0,0 +1,21 @@ + Copyright (c) 2004-2017 QOS.ch + 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. diff --git a/x-pack/plugin/security/licenses/slf4j-api-NOTICE.txt b/x-pack/plugin/security/licenses/slf4j-api-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/x-pack/plugin/vector-tile/licenses/slf4j-api-LICENSE.txt b/x-pack/plugin/vector-tile/licenses/slf4j-api-LICENSE.txt new file mode 100644 index 0000000000000..c8e72c2cd4738 --- /dev/null +++ b/x-pack/plugin/vector-tile/licenses/slf4j-api-LICENSE.txt @@ -0,0 +1,21 @@ + Copyright (c) 2004-2017 QOS.ch + 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. diff --git a/x-pack/plugin/vector-tile/licenses/slf4j-api-NOTICE.txt b/x-pack/plugin/vector-tile/licenses/slf4j-api-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d From 8cca7e2dbe5108a90ab1afe81c25e27b388abe89 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Fri, 21 Mar 2025 17:06:17 +0000 Subject: [PATCH 14/19] [CI] Auto commit changes from spotless --- .../elasticsearch/arrow/bulk/ArrowBulkAction.java | 4 +++- .../arrow/bulk/ArrowIncrementalParser.java | 2 +- .../arrow/xcontent/ArrowToXContentTests.java | 5 +---- .../rest/action/document/RestBulkAction.java | 13 ++++++++++--- 4 files changed, 15 insertions(+), 9 deletions(-) diff --git a/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkAction.java b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkAction.java index f6b557a70c7ae..d74591d3a168d 100644 --- a/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkAction.java +++ b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkAction.java @@ -58,7 +58,9 @@ protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient cli String waitForActiveShards = request.param("wait_for_active_shards"); TimeValue timeout = request.paramAsTime("timeout", BulkShardRequest.DEFAULT_TIMEOUT); String refresh = request.param("refresh"); - return new RestBulkAction.ChunkHandler(false, request, + return new RestBulkAction.ChunkHandler( + false, + request, () -> bulkHandler.newBulkRequest(waitForActiveShards, timeout, refresh), new ArrowBulkRequestParser(request) ); diff --git a/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowIncrementalParser.java b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowIncrementalParser.java index ef626e96cd30d..000180de2c3a1 100644 --- a/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowIncrementalParser.java +++ b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowIncrementalParser.java @@ -16,8 +16,8 @@ import org.apache.arrow.vector.ipc.ReadChannel; import org.apache.arrow.vector.ipc.message.MessageMetadataResult; import org.apache.arrow.vector.ipc.message.MessageSerializer; -import org.elasticsearch.libs.arrow.ArrowFormatException; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.libs.arrow.ArrowFormatException; import java.io.Closeable; import java.io.IOException; diff --git a/modules/arrow/src/test/java/org/elasticsearch/arrow/xcontent/ArrowToXContentTests.java b/modules/arrow/src/test/java/org/elasticsearch/arrow/xcontent/ArrowToXContentTests.java index 8632cef233dc1..ab1947a9f82c3 100644 --- a/modules/arrow/src/test/java/org/elasticsearch/arrow/xcontent/ArrowToXContentTests.java +++ b/modules/arrow/src/test/java/org/elasticsearch/arrow/xcontent/ArrowToXContentTests.java @@ -37,10 +37,7 @@ private static void checkPosition(ValueVector vector, int position, String json) public void testWriteField() throws IOException { - try ( - var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); - IntVector vector = new IntVector("intField", allocator); - ) { + try (var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); IntVector vector = new IntVector("intField", allocator);) { vector.allocateNew(1); vector.set(0, 123); vector.setValueCount(1); diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java index 4d90d3c375cc7..cd382090b4f3d 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java @@ -160,13 +160,20 @@ public static class ChunkHandler implements BaseRestHandler.RequestBodyChunkCons private final ArrayList> items = new ArrayList<>(4); public ChunkHandler(boolean allowExplicitIndex, RestRequest request, Supplier handlerSupplier) { - this(allowExplicitIndex, request, handlerSupplier, + this( + allowExplicitIndex, + request, + handlerSupplier, new BulkRequestParser(true, RestUtils.getIncludeSourceOnError(request), request.getRestApiVersion()) ); } - public ChunkHandler(boolean allowExplicitIndex, RestRequest request, Supplier handlerSupplier, - AbstractBulkRequestParser requestParser) { + public ChunkHandler( + boolean allowExplicitIndex, + RestRequest request, + Supplier handlerSupplier, + AbstractBulkRequestParser requestParser + ) { this.request = request; this.handlerSupplier = handlerSupplier; this.parser = requestParser.incrementalParser( From 86ab39020dfffc79de91e20748fa9072fc155132 Mon Sep 17 00:00:00 2001 From: Sylvain Wallez Date: Fri, 21 Mar 2025 20:49:29 +0100 Subject: [PATCH 15/19] Move Arrow-related security policy to the Arrow module --- .../plugin-metadata/plugin-security.policy | 24 +++++++++++++++++++ .../elasticsearch/bootstrap/security.policy | 12 ---------- 2 files changed, 24 insertions(+), 12 deletions(-) create mode 100644 modules/arrow/src/main/plugin-metadata/plugin-security.policy diff --git a/modules/arrow/src/main/plugin-metadata/plugin-security.policy b/modules/arrow/src/main/plugin-metadata/plugin-security.policy new file mode 100644 index 0000000000000..eb16f071ae2dd --- /dev/null +++ b/modules/arrow/src/main/plugin-metadata/plugin-security.policy @@ -0,0 +1,24 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +// Arrow uses Unsafe to access memory and accesses a private field in java.nio.Buffer +// See also additional global permissions below +grant codebase "${codebase.arrow-memory-core}" { + permission java.lang.RuntimePermission "accessDeclaredMembers"; + permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; + permission java.lang.RuntimePermission "accessClassInPackage.sun.misc"; +}; + +//// Everything else: +grant { + // This should be covered by arrow-memory-core above, but field access to java.nio.Buffer.address + // from org.apache.arrow.memory.util.MemoryUtil isn't in a privileged section. + permission java.lang.RuntimePermission "accessDeclaredMembers"; + permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; +}; diff --git a/server/src/main/resources/org/elasticsearch/bootstrap/security.policy b/server/src/main/resources/org/elasticsearch/bootstrap/security.policy index 39be2ca49eab5..17c682319c98f 100644 --- a/server/src/main/resources/org/elasticsearch/bootstrap/security.policy +++ b/server/src/main/resources/org/elasticsearch/bootstrap/security.policy @@ -73,22 +73,10 @@ grant codeBase "${codebase.elasticsearch-simdvec}" { permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; }; -// Arrow uses Unsafe to access memory and accesses a private field in java.nio.Buffer -// See also additional global permissions below -grant codebase "${codebase.arrow-memory-core}" { - permission java.lang.RuntimePermission "accessDeclaredMembers"; - permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; - permission java.lang.RuntimePermission "accessClassInPackage.sun.misc"; -}; //// Everything else: grant { - // This should be covered by arrow-memory-core above, but field access to java.nio.Buffer.address - // from org.apache.arrow.memory.util.MemoryUtil isn't in a privileged section. - permission java.lang.RuntimePermission "accessDeclaredMembers"; - permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; - // needed by vendored Guice permission java.lang.RuntimePermission "accessClassInPackage.jdk.internal.vm.annotation"; From 44a1b64506c53faa10e22c758221e48d47a9f6c3 Mon Sep 17 00:00:00 2001 From: Sylvain Wallez Date: Sat, 22 Mar 2025 14:15:47 +0100 Subject: [PATCH 16/19] Fix API name, remove test warning --- .../gradle/internal/ElasticsearchTestBasePlugin.java | 3 ++- .../src/main/resources/rest-api-spec/api/arrow.bulk.json | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/ElasticsearchTestBasePlugin.java b/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/ElasticsearchTestBasePlugin.java index 6bbd3d31d5030..99e2cb72c7d18 100644 --- a/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/ElasticsearchTestBasePlugin.java +++ b/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/ElasticsearchTestBasePlugin.java @@ -120,7 +120,8 @@ public void execute(Task t) { "--add-opens=java.base/java.nio.file=ALL-UNNAMED", "--add-opens=java.base/java.time=ALL-UNNAMED", "--add-opens=java.management/java.lang.management=ALL-UNNAMED", - "--add-opens=java.base/java.nio=org.apache.arrow.memory.core,ALL-UNNAMED", + // Arrow (may need to be replaced by org.apache.arrow.memory.core once modularized) + "--add-opens=java.base/java.nio=ALL-UNNAMED", // Define the allocation manager type to avoid classpath scanning to locate one. "-Darrow.allocation.manager.type=Unsafe", "-XX:+HeapDumpOnOutOfMemoryError" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/arrow.bulk.json b/rest-api-spec/src/main/resources/rest-api-spec/api/arrow.bulk.json index a13b59ff78cd9..c7ebc9bb7c114 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/arrow.bulk.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/arrow.bulk.json @@ -1,5 +1,5 @@ { - "bulk":{ + "arrow.bulk":{ "documentation":{ "url":"https://www.elastic.co/guide/en/elasticsearch/reference/master/docs-bulk.html", "description":"Allows to perform multiple index/update/delete operations in a single request using the Arrow IPC streaming format." From 181e78a9babf1ff64474b059ec66aea566bb51a4 Mon Sep 17 00:00:00 2001 From: Sylvain Wallez Date: Sun, 23 Mar 2025 16:59:24 +0100 Subject: [PATCH 17/19] Re-add tests that were lost in the refactoring --- modules/arrow/build.gradle | 5 +- .../arrow/bulk/ArrowBulkActionIT.java | 134 +++++ .../arrow/bulk/ESSingleNodeRestTestCase.java | 42 ++ .../bulk/ArrowBulkIncrementalParserTests.java | 466 ++++++++++++++++++ .../bulk/BytesReferenceChannelTests.java | 50 ++ .../arrow/xcontent/ArrowToXContentTests.java | 9 +- 6 files changed, 696 insertions(+), 10 deletions(-) create mode 100644 modules/arrow/src/internalClusterTest/java/org/elasticsearch/arrow/bulk/ArrowBulkActionIT.java create mode 100644 modules/arrow/src/internalClusterTest/java/org/elasticsearch/arrow/bulk/ESSingleNodeRestTestCase.java create mode 100644 modules/arrow/src/test/java/org/elasticsearch/arrow/bulk/ArrowBulkIncrementalParserTests.java create mode 100644 modules/arrow/src/test/java/org/elasticsearch/arrow/bulk/BytesReferenceChannelTests.java diff --git a/modules/arrow/build.gradle b/modules/arrow/build.gradle index ae394fef4512f..403d88b22cd0b 100644 --- a/modules/arrow/build.gradle +++ b/modules/arrow/build.gradle @@ -7,16 +7,13 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ -import org.elasticsearch.gradle.internal.precommit.CheckForbiddenApisTask - apply plugin: 'elasticsearch.internal-es-plugin' -apply plugin: 'elasticsearch.internal-java-rest-test' +apply plugin: 'elasticsearch.internal-cluster-test' esplugin { name = 'arrow' description = 'Provides Arrow integration for Elasticsearch' classname ='org.elasticsearch.arrow.ArrowPlugin' - //extendedPlugins = ['lang-painless'] } dependencies { diff --git a/modules/arrow/src/internalClusterTest/java/org/elasticsearch/arrow/bulk/ArrowBulkActionIT.java b/modules/arrow/src/internalClusterTest/java/org/elasticsearch/arrow/bulk/ArrowBulkActionIT.java new file mode 100644 index 0000000000000..04fb4d9679375 --- /dev/null +++ b/modules/arrow/src/internalClusterTest/java/org/elasticsearch/arrow/bulk/ArrowBulkActionIT.java @@ -0,0 +1,134 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.arrow.bulk; + +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.ipc.ArrowStreamWriter; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.FieldType; +import org.apache.arrow.vector.types.pojo.Schema; +import org.apache.arrow.vector.util.Text; +import org.apache.http.entity.ByteArrayEntity; +import org.apache.http.entity.ContentType; +import org.elasticsearch.arrow.ArrowPlugin; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.libs.arrow.Arrow; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xcontent.XContentType; +import org.junit.After; +import org.junit.Before; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +/** + * End-to-end test for Arrow bulk ingestion. Tests for the various Arrow datatypes and + * bulk actions are in {@code ArrowBulkIncrementalParserTests} + */ +public class ArrowBulkActionIT extends ESSingleNodeRestTestCase { + + private RestClient restClient; + + @Before + public void init() { + restClient = createRestClient(); + } + + @After + public void cleanup() throws IOException { + restClient.close(); + } + + @Override + protected Collection> getPlugins() { + return Collections.singletonList(ArrowPlugin.class); + } + + /** + * An end-to-end test that checks that Arrow data is correctly indexed and can be searched. + */ + public void testBulk() throws Exception { + + String index = "arrow_bulk_test"; + + { + // Check that the index doesn't exist + var request = new Request("HEAD", "/" + index); + var response = restClient.performRequest(request); + assertEquals(404, response.getStatusLine().getStatusCode()); + } + + // Create a dataframe with two columns: integer and string + Field intField = new Field("ints", FieldType.nullable(new ArrowType.Int(32, true)), null); + Field strField = new Field("strings", FieldType.nullable(new ArrowType.Utf8()), null); + Schema schema = new Schema(List.of(intField, strField)); + + int batchCount = randomInt(10); + int rowCount = randomInt(10); + + byte[] payload; + + // Create vectors and write them to a byte array + try (var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); var root = VectorSchemaRoot.create(schema, allocator);) { + var baos = new ByteArrayOutputStream(); + IntVector intVector = (IntVector) root.getVector(0); + VarCharVector stringVector = (VarCharVector) root.getVector(1); + + try (ArrowStreamWriter writer = new ArrowStreamWriter(root, null, baos)) { + for (int batch = 0; batch < batchCount; batch++) { + intVector.allocateNew(rowCount); + stringVector.allocateNew(rowCount); + for (int row = 0; row < rowCount; row++) { + int globalRow = row + batch * rowCount; + intVector.set(row, globalRow); + stringVector.set(row, new Text("row" + globalRow)); + } + root.setRowCount(rowCount); + writer.writeBatch(); + } + } + payload = baos.toByteArray(); + } + + { + // Bulk insert the arrow stream + var request = new Request("POST", "/_arrow/" + index + "/_bulk"); + request.addParameter("refresh", "wait_for"); + request.setEntity(new ByteArrayEntity(payload, ContentType.create(Arrow.MEDIA_TYPE))); + + var response = restClient.performRequest(request); + var result = XContentType.JSON.xContent() + .createParser(XContentParserConfiguration.EMPTY, response.getEntity().getContent()) + .map(); + + assertEquals(Boolean.FALSE, result.get("errors")); + assertEquals(batchCount * rowCount, ((List) result.get("items")).size()); + } + + { + // Check that the index effectively contains what we sent + var request = new Request("GET", "/" + index + "/_count"); + var response = restClient.performRequest(request); + var result = XContentType.JSON.xContent() + .createParser(XContentParserConfiguration.EMPTY, response.getEntity().getContent()) + .map(); + + assertEquals(batchCount * rowCount, result.get("count")); + } + } +} diff --git a/modules/arrow/src/internalClusterTest/java/org/elasticsearch/arrow/bulk/ESSingleNodeRestTestCase.java b/modules/arrow/src/internalClusterTest/java/org/elasticsearch/arrow/bulk/ESSingleNodeRestTestCase.java new file mode 100644 index 0000000000000..5d33862db9d74 --- /dev/null +++ b/modules/arrow/src/internalClusterTest/java/org/elasticsearch/arrow/bulk/ESSingleNodeRestTestCase.java @@ -0,0 +1,42 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.arrow.bulk; + +import org.apache.http.HttpHost; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.common.network.NetworkAddress; +import org.elasticsearch.http.HttpInfo; +import org.elasticsearch.node.NodeService; +import org.elasticsearch.test.ESSingleNodeTestCase; + +import java.net.InetSocketAddress; + +/** + * An {@link ESSingleNodeTestCase} with a Rest client (a feature that is provided by {@code ESIntegTestCase}). + */ +public abstract class ESSingleNodeRestTestCase extends ESSingleNodeTestCase { + + @Override + protected boolean addMockHttpTransport() { + return false; + } + + public RestClient createRestClient() { + NodeService instance = node().injector().getInstance(NodeService.class); + var httpInfo = instance.info(false, false, false, false, false, false, true, false, false, false, false, false) + .getInfo(HttpInfo.class); + + assertNotNull("Couldn't get the node's http info", httpInfo); + + InetSocketAddress address = httpInfo.address().publishAddress().address(); + HttpHost host = new HttpHost(NetworkAddress.format(address.getAddress()), address.getPort(), "http"); + return RestClient.builder(host).build(); + } +} diff --git a/modules/arrow/src/test/java/org/elasticsearch/arrow/bulk/ArrowBulkIncrementalParserTests.java b/modules/arrow/src/test/java/org/elasticsearch/arrow/bulk/ArrowBulkIncrementalParserTests.java new file mode 100644 index 0000000000000..0e95766f6e2c2 --- /dev/null +++ b/modules/arrow/src/test/java/org/elasticsearch/arrow/bulk/ArrowBulkIncrementalParserTests.java @@ -0,0 +1,466 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.arrow.bulk; + +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.complex.MapVector; +import org.apache.arrow.vector.complex.StructVector; +import org.apache.arrow.vector.dictionary.Dictionary; +import org.apache.arrow.vector.dictionary.DictionaryEncoder; +import org.apache.arrow.vector.dictionary.DictionaryProvider; +import org.apache.arrow.vector.ipc.ArrowStreamWriter; +import org.apache.arrow.vector.types.Types; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.DictionaryEncoding; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.FieldType; +import org.apache.arrow.vector.types.pojo.Schema; +import org.apache.arrow.vector.util.Text; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.core.RestApiVersion; +import org.elasticsearch.libs.arrow.Arrow; +import org.elasticsearch.search.fetch.subphase.FetchSourceContext; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xcontent.XContentType; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.function.BiConsumer; +import java.util.function.Consumer; + +import static org.hamcrest.Matchers.equalTo; + +public class ArrowBulkIncrementalParserTests extends ESTestCase { + + // ----- Test Arrow batches and incremental parsing + + public void testBatchingAndChunking() throws IOException { + checkBatchingAndChunking(1, 10, false); + checkBatchingAndChunking(1, 10, true); + checkBatchingAndChunking(2, 10, false); + checkBatchingAndChunking(2, 10, true); + } + + /** Create a payload for a 1-column dataframe (int and string), given a number of batches and rows per batch */ + private void checkBatchingAndChunking(int batchCount, int rowCount, boolean incremental) throws IOException { + byte[] payload; + + // Create a dataframe with two columns: integer and string + Field intField = new Field("ints", FieldType.nullable(new ArrowType.Int(32, true)), null); + Field strField = new Field("strings", FieldType.nullable(new ArrowType.Utf8()), null); + Schema schema = new Schema(List.of(intField, strField)); + + // Create vectors and write them to a byte array + try (var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); var root = VectorSchemaRoot.create(schema, allocator);) { + var baos = new ByteArrayOutputStream(); + IntVector intVector = (IntVector) root.getVector(0); + VarCharVector stringVector = (VarCharVector) root.getVector(1); + + try (ArrowStreamWriter writer = new ArrowStreamWriter(root, null, baos)) { + for (int batch = 0; batch < batchCount; batch++) { + intVector.allocateNew(rowCount); + stringVector.allocateNew(rowCount); + for (int row = 0; row < rowCount; row++) { + int globalRow = row + batch * rowCount; + intVector.set(row, globalRow); + stringVector.set(row, new Text("row" + globalRow)); + } + root.setRowCount(rowCount); + writer.writeBatch(); + } + } + payload = baos.toByteArray(); + } + + var operations = new ArrayList>(); + try (var parser = createParser("test", operations)) { + parse(parser, payload, incremental); + } + ; + + assertEquals(batchCount * rowCount, operations.size()); + + for (int i = 0; i < operations.size(); i++) { + IndexRequest operation = (IndexRequest) operations.get(i); + + assertEquals(DocWriteRequest.OpType.INDEX, operation.opType()); + assertEquals("test", operation.index()); + + assertEquals(XContentType.CBOR, operation.getContentType()); + + var map = operation.sourceAsMap(); + assertEquals(i, map.get("ints")); + assertEquals("row" + i, map.get("strings")); + } + } + + public void testInlineIdAndIndex() throws Exception { + byte[] payload; + + Field indexField = new Field("_index", FieldType.nullable(new ArrowType.Utf8()), null); + Field idField = new Field("_id", FieldType.nullable(new ArrowType.Utf8()), null); + Field intField = new Field("ints", FieldType.nullable(new ArrowType.Int(32, true)), null); + Field strField = new Field("strings", FieldType.nullable(new ArrowType.Utf8()), null); + Schema schema = new Schema(List.of(indexField, idField, intField, strField)); + + try (var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); var root = VectorSchemaRoot.create(schema, allocator);) { + var baos = new ByteArrayOutputStream(); + VarCharVector indexVector = (VarCharVector) root.getVector(0); + VarCharVector idVector = (VarCharVector) root.getVector(1); + IntVector intVector = (IntVector) root.getVector(2); + VarCharVector stringVector = (VarCharVector) root.getVector(3); + + try (ArrowStreamWriter writer = new ArrowStreamWriter(root, null, baos)) { + indexVector.allocateNew(4); + idVector.allocateNew(4); + intVector.allocateNew(4); + stringVector.allocateNew(4); + + // No index, no id + indexVector.setNull(0); + idVector.setNull(0); + stringVector.set(0, new Text("row0")); + intVector.set(0, 0); + + // No index, id + indexVector.setNull(1); + idVector.set(1, new Text("id1")); + stringVector.set(1, new Text("row1")); + intVector.set(1, 1); + + // Index, no id + indexVector.set(2, new Text("index2")); + idVector.setNull(2); + stringVector.set(2, new Text("row2")); + intVector.set(2, 2); + + // Index & id + indexVector.set(3, new Text("index3")); + idVector.set(3, new Text("id3")); + stringVector.set(1, new Text("row3")); + intVector.set(1, 3); + + root.setRowCount(4); + writer.writeBatch(); + } + payload = baos.toByteArray(); + } + + var operations = new ArrayList>(); + try (var parser = createParser("defaultIndex", operations)) { + parse(parser, payload, false); + } + ; + + IndexRequest operation = (IndexRequest) operations.get(0); + assertEquals("defaultIndex", operation.index()); + assertEquals(null, operation.id()); + + operation = (IndexRequest) operations.get(1); + assertEquals("defaultIndex", operation.index()); + assertEquals("id1", operation.id()); + + operation = (IndexRequest) operations.get(2); + assertEquals("index2", operation.index()); + assertEquals(null, operation.id()); + + operation = (IndexRequest) operations.get(3); + assertEquals("index3", operation.index()); + assertEquals("id3", operation.id()); + + } + + // ----- Test action decoding + + /** Action as a map of (string, string) */ + public void testActionsAsStringMap() throws Exception { + + try ( + var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); + var vector = new MapVector("action", allocator, FieldType.nullable(new ArrowType.Map(false)), null); + var parser = createParser("default-index", List.of()) + ) { + var w = vector.getWriter(); + + w.startMap(); + + // Override operation type (default is create) + w.startEntry(); + w.key().varChar().writeVarChar("op_type"); + w.value().varChar().writeVarChar("update"); + w.endEntry(); + + // Override default "default-index" index + w.startEntry(); + w.key().varChar().writeVarChar("_index"); + w.value().varChar().writeVarChar("first-index"); + w.endEntry(); + + // Set if_seq_no as a string, to test a lazy approach with a simple (string, string) map + w.startEntry(); + w.key().varChar().writeVarChar("if_seq_no"); + w.value().varChar().writeVarChar("3"); + w.endEntry(); + + w.endMap(); + + w.startMap(); + + // Override default "default-index" index + w.startEntry(); + w.key().varChar().writeVarChar("_index"); + w.value().varChar().writeVarChar("second-index"); + w.endEntry(); + + // Override operation type (default is create) + w.startEntry(); + w.key().varChar().writeVarChar("op_type"); + w.value().varChar().writeVarChar("index"); + w.endEntry(); + + // Set version as a string, to test a lazy approach with a simple (string, string) map + w.startEntry(); + w.key().varChar().writeVarChar("if_seq_no"); + w.value().varChar().writeVarChar("4"); + w.endEntry(); + + w.endMap(); + + vector.setValueCount(w.getPosition()); + // Value type is varchar + assertEquals(Types.MinorType.VARCHAR, vector.getChildrenFromFields().get(0).getChildrenFromFields().get(1).getMinorType()); + + { + var request = parser.parseAction(vector, 0, null, null); + assertEquals(DocWriteRequest.OpType.UPDATE, request.opType()); + assertEquals("first-index", request.index()); + assertEquals(3, request.ifSeqNo()); + } + + { + var request = parser.parseAction(vector, 1, null, null); + assertEquals(DocWriteRequest.OpType.INDEX, request.opType()); + assertEquals("second-index", request.index()); + assertEquals(4, request.ifSeqNo()); + } + } + } + + /** Action as a map of (string, union(string, int)) */ + public void testActionsAsUnionMap() throws Exception { + + try ( + var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); + var vector = new MapVector("action", allocator, FieldType.nullable(new ArrowType.Map(false)), null); + var parser = createParser("default-index", List.of()) + ) { + var w = vector.getWriter(); + + w.startMap(); + + // Override operation type (default is create) + w.startEntry(); + w.key().varChar().writeVarChar("op_type"); + w.value().varChar().writeVarChar("update"); + w.endEntry(); + + // Override default "default-index" index + w.startEntry(); + w.key().varChar().writeVarChar("_index"); + w.value().varChar().writeVarChar("some-index"); + w.endEntry(); + + // Set version as a number. This promotes the value field to a union type + w.startEntry(); + w.key().varChar().writeVarChar("if_seq_no"); + w.value().integer().writeInt(3); + w.endEntry(); + + w.endMap(); + + vector.setValueCount(w.getPosition()); + var request = parser.parseAction(vector, 0, null, null); + + // Value type is a union + assertEquals(Types.MinorType.UNION, vector.getChildrenFromFields().get(0).getChildrenFromFields().get(1).getMinorType()); + + assertEquals(DocWriteRequest.OpType.UPDATE, request.opType()); + assertEquals("some-index", request.index()); + assertEquals(3, request.ifSeqNo()); + } + } + + /** Action as a struct */ + public void testActionsAsStruct() throws Exception { + + try ( + var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); + var vector = new StructVector("action", allocator, FieldType.nullable(new ArrowType.Struct()), null); + var parser = createParser("default-index", List.of()) + ) { + var w = vector.getWriter(); + + w.start(); + w.varChar("op_type").writeVarChar("update"); + w.varChar("_index").writeVarChar("first-index"); + w.integer("if_seq_no").writeInt(3); + w.end(); + + w.start(); + w.varChar("op_type").writeVarChar("index"); + w.varChar("_index").writeVarChar("second-index"); + w.integer("if_seq_no").writeInt(4); + w.end(); + + vector.setValueCount(w.getPosition()); + + { + var request = parser.parseAction(vector, 0, null, null); + assertEquals(DocWriteRequest.OpType.UPDATE, request.opType()); + assertEquals("first-index", request.index()); + assertEquals(3, request.ifSeqNo()); + } + + { + var request = parser.parseAction(vector, 1, null, null); + assertEquals(DocWriteRequest.OpType.INDEX, request.opType()); + assertEquals("second-index", request.index()); + assertEquals(4, request.ifSeqNo()); + } + } + } + + // ----- Dictionary encoding + public void testDictionaryEncoding() throws Exception { + + ByteArrayOutputStream payload = new ByteArrayOutputStream(); + + try ( + var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); + VarCharVector dictVector = new VarCharVector("dict", allocator); + VarCharVector vector = new VarCharVector("data_field", allocator); + DictionaryProvider.MapDictionaryProvider dictionaryProvider = new DictionaryProvider.MapDictionaryProvider(); + ) { + // create dictionary lookup vector + dictVector.allocateNewSafe(); + dictVector.setSafe(0, new Text("aa")); + dictVector.setSafe(1, new Text("bb")); + dictVector.setSafe(2, new Text("cc")); + dictVector.setValueCount(3); + + // create dictionary + long dictionaryId = 1L; + Dictionary dictionary = new Dictionary(dictVector, new DictionaryEncoding(dictionaryId, false, /*indexType=*/null)); + + dictionaryProvider.put(dictionary); + + // create original data vector + vector.allocateNewSafe(); + vector.setSafe(0, new Text("bb")); + vector.setSafe(1, new Text("bb")); + vector.setSafe(2, new Text("cc")); + vector.setSafe(3, new Text("aa")); + vector.setValueCount(4); + + // Encode the vector with the dictionary + IntVector encodedVector = (IntVector) DictionaryEncoder.encode(vector, dictionary); + + // create VectorSchemaRoot + List fields = List.of(encodedVector.getField()); + List vectors = List.of(encodedVector); + + try ( + VectorSchemaRoot root = new VectorSchemaRoot(fields, vectors); + ArrowStreamWriter writer = new ArrowStreamWriter(root, dictionaryProvider, payload); + ) { + // write data + writer.start(); + writer.writeBatch(); + writer.end(); + } + + var operations = new ArrayList>(); + try (var parser = createParser("defaultIndex", operations)) { + parse(parser, payload.toByteArray(), false); + } + + // Check that dictionary-encoded values were correctly decoded + assertEquals("bb", ((IndexRequest) operations.get(0)).sourceAsMap().get("data_field")); + assertEquals("bb", ((IndexRequest) operations.get(1)).sourceAsMap().get("data_field")); + assertEquals("cc", ((IndexRequest) operations.get(2)).sourceAsMap().get("data_field")); + assertEquals("aa", ((IndexRequest) operations.get(3)).sourceAsMap().get("data_field")); + } + } + + // ----- Utilities + + private static ArrowBulkIncrementalParser createParser(String defaultIndex, List> requests) { + + DocWriteRequest.OpType defaultOpType = DocWriteRequest.OpType.INDEX; + String defaultRouting = null; + FetchSourceContext defaultFetchSourceContext = null; + String defaultPipeline = null; + Boolean defaultRequireAlias = false; + Boolean defaultRequireDataStream = false; + Boolean defaultListExecutedPipelines = false; + + boolean allowExplicitIndex = true; + XContentType xContentType = null; + BiConsumer indexRequestConsumer = (r, t) -> requests.add(r); + Consumer updateRequestConsumer = requests::add; + Consumer deleteRequestConsumer = requests::add; + + return new ArrowBulkIncrementalParser( + defaultOpType, + defaultIndex, + defaultRouting, + defaultFetchSourceContext, + defaultPipeline, + defaultRequireAlias, + defaultRequireDataStream, + defaultListExecutedPipelines, + allowExplicitIndex, + xContentType, + XContentParserConfiguration.EMPTY.withRestApiVersion(RestApiVersion.current()), + indexRequestConsumer, + updateRequestConsumer, + deleteRequestConsumer + ); + } + + private void parse(ArrowBulkIncrementalParser parser, byte[] payload, boolean incremental) throws IOException { + + int consumed = 0; + var request = new BytesArray(payload); + + if (incremental) { + // Borrowed from BulkRequestParserTests + for (int i = 0; i < request.length() - 1; ++i) { + consumed += parser.parse(request.slice(consumed, i - consumed + 1), false); + } + consumed += parser.parse(request.slice(consumed, request.length() - consumed), true); + assertThat(consumed, equalTo(request.length())); + } else { + consumed = parser.parse(request, true); + } + + assertEquals(payload.length, consumed); + } +} diff --git a/modules/arrow/src/test/java/org/elasticsearch/arrow/bulk/BytesReferenceChannelTests.java b/modules/arrow/src/test/java/org/elasticsearch/arrow/bulk/BytesReferenceChannelTests.java new file mode 100644 index 0000000000000..0381b707bd8c9 --- /dev/null +++ b/modules/arrow/src/test/java/org/elasticsearch/arrow/bulk/BytesReferenceChannelTests.java @@ -0,0 +1,50 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.arrow.bulk; + +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.bytes.CompositeBytesReference; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.nio.channels.Channels; +import java.util.ArrayList; + +public class BytesReferenceChannelTests extends ESTestCase { + + /** + * Check iteration on the buffers of a composite byteref + */ + public void testMultipleBuffers() throws IOException { + + var chunks = new ArrayList(); + byte[] chunk = null; + for (int i = 0; i < 0x100; i++) { + if (i % 0x10 == 0) { + chunk = new byte[0x10]; + chunks.add(new BytesArray(chunk)); + } + chunk[i % 0x10] = (byte) i; + } + + var bytesref = CompositeBytesReference.of(chunks.toArray(new BytesReference[0])); + + try (var channel = new BytesReferenceChannel(bytesref)) { + var in = Channels.newInputStream(channel); + + for (int i = 0; i < 0x100; i++) { + assertEquals(i, in.read()); + } + + assertEquals(-1, in.read()); + } + } +} diff --git a/modules/arrow/src/test/java/org/elasticsearch/arrow/xcontent/ArrowToXContentTests.java b/modules/arrow/src/test/java/org/elasticsearch/arrow/xcontent/ArrowToXContentTests.java index ab1947a9f82c3..70e504ab232bf 100644 --- a/modules/arrow/src/test/java/org/elasticsearch/arrow/xcontent/ArrowToXContentTests.java +++ b/modules/arrow/src/test/java/org/elasticsearch/arrow/xcontent/ArrowToXContentTests.java @@ -48,7 +48,7 @@ public void testWriteField() throws IOException { public void testWriteVarChar() throws Exception { try ( - var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); + var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); VarCharVector vector = new VarCharVector("stringField", allocator); ) { vector.allocateNew(); @@ -61,7 +61,7 @@ public void testWriteVarChar() throws Exception { public void testWriteMap() throws Exception { try ( - var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); + var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); MapVector vector = MapVector.empty("mapField", allocator, false); ) { var w = vector.getWriter(); @@ -78,10 +78,7 @@ public void testWriteMap() throws Exception { } public void testWriteNullValue() throws Exception { - try ( - var allocator = Arrow.rootAllocator().newChildAllocator("test", 0, Long.MAX_VALUE); - IntVector vector = new IntVector("intField", allocator); - ) { + try (var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); IntVector vector = new IntVector("intField", allocator);) { vector.allocateNew(1); vector.setNull(0); vector.setValueCount(1); From d686e885b54465760de4af8db5d9c1b354b9c40b Mon Sep 17 00:00:00 2001 From: Sylvain Wallez Date: Sat, 12 Apr 2025 11:31:47 +0200 Subject: [PATCH 18/19] Add support for timestamps, more tests --- .../bulk/ArrowBulkIncrementalParser.java | 55 ++-- .../bulk/BytesReferenceOutputStream.java | 23 ++ .../arrow/xcontent/ArrowToXContent.java | 162 ++++++++-- .../bulk/ArrowBulkIncrementalParserTests.java | 1 - .../arrow/xcontent/ArrowToXContentTests.java | 292 ++++++++++++++++-- 5 files changed, 446 insertions(+), 87 deletions(-) create mode 100644 modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/BytesReferenceOutputStream.java diff --git a/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkIncrementalParser.java b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkIncrementalParser.java index b5f34df43b6c6..7f91bc40a1459 100644 --- a/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkIncrementalParser.java +++ b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkIncrementalParser.java @@ -10,7 +10,6 @@ package org.elasticsearch.arrow.bulk; import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.memory.RootAllocator; import org.apache.arrow.vector.BaseIntVector; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.ValueVector; @@ -30,7 +29,6 @@ import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.arrow.xcontent.ArrowToXContent; import org.elasticsearch.arrow.xcontent.XContentBuffer; -import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.core.Nullable; import org.elasticsearch.libs.arrow.Arrow; @@ -40,7 +38,6 @@ import org.elasticsearch.xcontent.XContentParserConfiguration; import org.elasticsearch.xcontent.XContentType; -import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.Arrays; @@ -54,14 +51,9 @@ class ArrowBulkIncrementalParser extends BulkRequestParser.XContentIncrementalPa /** XContent format used to encode source documents */ private static final XContent SOURCE_XCONTENT = XContentType.CBOR.xContent(); - private static final String ID = "_id"; - private static final String INDEX = "_index"; - private static final String ACTION = "_bulk_action"; + private final DocWriteRequest.OpType defaultOpType; - private DocWriteRequest.OpType defaultOpType; - - private ArrowIncrementalParser arrowParser; - private BufferAllocator allocator; + private final ArrowIncrementalParser arrowParser; private VectorSchemaRoot schemaRoot; private Map dictionaries; @@ -70,6 +62,8 @@ class ArrowBulkIncrementalParser extends BulkRequestParser.XContentIncrementalPa private Integer actionField = null; private BitSet valueFields; + private final ArrowToXContent arrowToXContent = new ArrowToXContent(); + ArrowBulkIncrementalParser( DocWriteRequest.OpType defaultOpType, @Nullable String defaultIndex, @@ -106,9 +100,9 @@ class ArrowBulkIncrementalParser extends BulkRequestParser.XContentIncrementalPa this.defaultOpType = defaultOpType; // FIXME: hard-coded limit to 100 MiB per record batch. Should we add an AllocationListener that calls ES memory management? - this.allocator = Arrow.newChildAllocator("bulk-ingestion", 0, 100 * 1024 * 1024); + BufferAllocator allocator = Arrow.newChildAllocator("bulk-ingestion", 0, 100 * 1024 * 1024); - this.arrowParser = new ArrowIncrementalParser(new RootAllocator(), new ArrowIncrementalParser.Listener() { + this.arrowParser = new ArrowIncrementalParser(allocator, new ArrowIncrementalParser.Listener() { @Override public void startStream(VectorSchemaRoot schemaRoot) throws IOException { startArrowStream(schemaRoot); @@ -151,12 +145,12 @@ private void startArrowStream(VectorSchemaRoot root) { var field = schemaFields.get(i); switch (field.getName()) { - case ID -> idField = i; - case INDEX -> indexField = i; - case ACTION -> { + case ArrowBulkAction.ID -> idField = i; + case ArrowBulkAction.INDEX -> indexField = i; + case ArrowBulkAction.ACTION -> { var type = field.getMinorType(); if (type != Types.MinorType.MAP && type != Types.MinorType.STRUCT) { - throw new ArrowFormatException("Field '" + ACTION + "' should be a map or a struct"); + throw new ArrowFormatException("Field '" + ArrowBulkAction.ACTION + "' should be a map or a struct"); } actionField = i; } @@ -206,7 +200,7 @@ protected BytesReference generateSource(int position) throws IOException { int rowCount = schemaRoot.getRowCount(); for (int i = 0; i < rowCount; i++) { if (valueFields.get(i)) { - ArrowToXContent.writeField(schemaRoot.getVector(i), position, dictionaries, generator); + arrowToXContent.writeField(schemaRoot.getVector(i), position, dictionaries, generator); } } generator.writeEndObject(); @@ -242,7 +236,7 @@ DocWriteRequest parseAction(@Nullable FieldVector actionVector, int position, // Note: the "op_type" property may also exist, but the action parser accepts it. generator.writeStartObject(); generator.writeFieldName(opType); - ArrowToXContent.writeValue(actionVector, position, dictionaries, generator); + arrowToXContent.writeValue(actionVector, position, dictionaries, generator); generator.writeEndObject(); } @@ -252,7 +246,13 @@ DocWriteRequest parseAction(@Nullable FieldVector actionVector, int position, if (id != null) { if (request.id() != null) { throw new ArrowFormatException( - "'" + ID + "' found both as top-level field and in '" + ACTION + "' at position [" + position + "]" + "'" + + ArrowBulkAction.ID + + "' found both as top-level field and in '" + + ArrowBulkAction.ACTION + + "' at position [" + + position + + "]" ); } @@ -268,7 +268,13 @@ DocWriteRequest parseAction(@Nullable FieldVector actionVector, int position, // Testing references on purpose to detect default index passed down to the request if (request.index() != defaultIndex) { throw new ArrowFormatException( - "'" + INDEX + "' found both as top-level field and in '" + ACTION + "' at position [" + position + "]" + "'" + + ArrowBulkAction.INDEX + + "' found both as top-level field and in '" + + ArrowBulkAction.ACTION + + "' at position [" + + position + + "]" ); } request.index(index); @@ -341,13 +347,4 @@ private String getString(ValueVector vector, int position) { } }; } - - /** - * A byte array stream that can be converted to {@code BytesReference} with zero copy. - */ - private static class BytesReferenceOutputStream extends ByteArrayOutputStream { - BytesArray asBytesReference() { - return new BytesArray(buf, 0, count); - } - } } diff --git a/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/BytesReferenceOutputStream.java b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/BytesReferenceOutputStream.java new file mode 100644 index 0000000000000..8f36b0911ac7a --- /dev/null +++ b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/BytesReferenceOutputStream.java @@ -0,0 +1,23 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.arrow.bulk; + +import org.elasticsearch.common.bytes.BytesArray; + +import java.io.ByteArrayOutputStream; + +/** + * A byte array stream that can be converted to {@code BytesReference} with zero copy. + */ +class BytesReferenceOutputStream extends ByteArrayOutputStream { + BytesArray asBytesReference() { + return new BytesArray(buf, 0, count); + } +} diff --git a/modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent/ArrowToXContent.java b/modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent/ArrowToXContent.java index 950cf5e3ef0bb..c1314c2e1103c 100644 --- a/modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent/ArrowToXContent.java +++ b/modules/arrow/src/main/java/org/elasticsearch/arrow/xcontent/ArrowToXContent.java @@ -13,7 +13,18 @@ import org.apache.arrow.vector.BitVector; import org.apache.arrow.vector.FixedSizeBinaryVector; import org.apache.arrow.vector.FloatingPointVector; -import org.apache.arrow.vector.TimeStampVector; +import org.apache.arrow.vector.TimeMicroVector; +import org.apache.arrow.vector.TimeMilliVector; +import org.apache.arrow.vector.TimeNanoVector; +import org.apache.arrow.vector.TimeSecVector; +import org.apache.arrow.vector.TimeStampMicroTZVector; +import org.apache.arrow.vector.TimeStampMicroVector; +import org.apache.arrow.vector.TimeStampMilliTZVector; +import org.apache.arrow.vector.TimeStampMilliVector; +import org.apache.arrow.vector.TimeStampNanoTZVector; +import org.apache.arrow.vector.TimeStampNanoVector; +import org.apache.arrow.vector.TimeStampSecTZVector; +import org.apache.arrow.vector.TimeStampSecVector; import org.apache.arrow.vector.ValueVector; import org.apache.arrow.vector.VarCharVector; import org.apache.arrow.vector.VariableWidthFieldVector; @@ -29,21 +40,26 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.time.Instant; +import java.time.ZoneId; +import java.time.ZoneOffset; import java.util.EnumSet; +import java.util.HashMap; import java.util.Map; /** * Utility methods to serialize Arrow dataframes to XContent events. *

- * Limitations: + * Limitations and caveats: *

    - *
  • time and timestamps are converted to milliseconds (no support for nanoseconds) + *
  • time and timestamps are converted to milliseconds or nanoseconds depending on their unit *
  • *
  • some types aren't implemented *
  • *
* * @see Arrow data types + * @see Arrow schema */ public class ArrowToXContent { @@ -53,6 +69,22 @@ public class ArrowToXContent { Types.MinorType.VIEWVARCHAR ); + static final long MILLIS_PER_SEC = 1_000L; + static final long NANOS_PER_SEC = 1_000_000_000L; + static final long NANOS_PER_MILLI = 1_000_000L; + static final long NANOS_PER_MICRO = 1_000L; + + private final Map zidCache = new HashMap<>(); + + final long getUTCOffsetSeconds(long millis, String tz) { + var tzId = zidCache.computeIfAbsent(tz, ZoneId::of); + if (tzId instanceof ZoneOffset zo) { + return zo.getTotalSeconds(); + } + var instant = Instant.ofEpochMilli(millis); + return tzId.getRules().getOffset(instant).getTotalSeconds(); + } + /** * Write a field and its value from an Arrow vector as XContent * @@ -61,7 +93,7 @@ public class ArrowToXContent { * @param dictionaries to look up values for dictionary-encoded vectors * @param generator XContent output */ - public static void writeField(ValueVector vector, int position, Map dictionaries, XContentGenerator generator) + public void writeField(ValueVector vector, int position, Map dictionaries, XContentGenerator generator) throws IOException { generator.writeFieldName(vector.getName()); writeValue(vector, position, dictionaries, generator); @@ -75,8 +107,9 @@ public static void writeField(ValueVector vector, int position, Map dictionaries, XContentGenerator generator) + public void writeValue(ValueVector vector, int position, Map dictionaries, XContentGenerator generator) throws IOException { + if (vector.isNull(position)) { generator.writeNull(); return; @@ -87,8 +120,15 @@ public static void writeValue(ValueVector vector, int position, Map { var bytesVector = (VariableWidthFieldVector) vector; @@ -130,7 +170,7 @@ public static void writeValue(ValueVector vector, int position, Map { var listVector = (BaseListVector) vector; @@ -146,35 +186,104 @@ public static void writeValue(ValueVector vector, int position, Map { + var tsVector = (TimeSecVector) vector; + generator.writeNumber(tsVector.get(position) * MILLIS_PER_SEC); // millisecs + yield null; + } + + case TIMEMILLI -> { + var tsVector = (TimeMilliVector) vector; + generator.writeNumber(tsVector.get(position)); // millisecs + yield null; + } + + case TIMEMICRO -> { + var tsVector = (TimeMicroVector) vector; + generator.writeNumber(tsVector.get(position) * NANOS_PER_MICRO); // nanosecs + yield null; + } + + case TIMENANO -> { + var tsVector = (TimeNanoVector) vector; + generator.writeNumber(tsVector.get(position)); // nanosecs + yield null; + } + + // ----- Timestamp + // + // From the spec: "Timestamp is a 64-bit signed integer representing an elapsed time since a + // fixed epoch, stored in either of four units: seconds, milliseconds, + // microseconds or nanoseconds, and is optionally annotated with a timezone. + // If a Timestamp column has no timezone value, its epoch is + // 1970-01-01 00:00:00 (January 1st 1970, midnight) in an *unknown* timezone." + // + // Arrow/Java uses different types for timestamps with a timezone (TIMESTAMPXXXTZ) and without + // a timezone (TIMESTAMPXXX). + // ES doesn't support timezones, so the TIMESTAMPXXXTZ are not supported. + + case TIMESTAMPSEC -> { + var tsVector = (TimeStampSecVector) vector; + generator.writeNumber(tsVector.get(position) * MILLIS_PER_SEC); // millisecs + yield null; + } + + case TIMESTAMPMILLI -> { + var tsVector = (TimeStampMilliVector) vector; + generator.writeNumber(tsVector.get(position)); // millisecs + yield null; + } + + case TIMESTAMPMICRO -> { + var tsVector = (TimeStampMicroVector) vector; + generator.writeNumber(tsVector.get(position) * NANOS_PER_MICRO); // nanosecs + yield null; + } + + case TIMESTAMPNANO -> { + var tsVector = (TimeStampNanoVector) vector; + generator.writeNumber(tsVector.get(position)); // nanosecs + yield null; + } - // Timestamps are the elapsed time since the Epoch, with an optional timezone that - // can be used for timezome-aware operations or display. Since ES date fields - // don't support timezones, we ignore it. - // See https://github.com/apache/arrow/blob/main/format/Schema.fbs - // and https://www.elastic.co/guide/en/elasticsearch/reference/current/date.html + // ----- Timestamp with a timezone - case TIMESEC, TIMESTAMPSEC -> { - var tsVector = (TimeStampVector) vector; - generator.writeNumber(tsVector.get(position) * 1000); + case TIMESTAMPSECTZ -> { + var tsVector = (TimeStampSecTZVector) vector; + long millis = tsVector.get(position) * MILLIS_PER_SEC; + millis -= getUTCOffsetSeconds(millis, tsVector.getTimeZone()) * MILLIS_PER_SEC; + generator.writeNumber(millis); yield null; } - case TIMEMILLI, TIMESTAMPMILLI -> { - var tsVector = (TimeStampVector) vector; - generator.writeNumber(tsVector.get(position)); + case TIMESTAMPMILLITZ -> { + var tsVector = (TimeStampMilliTZVector) vector; + long millis = tsVector.get(position); + millis -= getUTCOffsetSeconds(millis, tsVector.getTimeZone()) * MILLIS_PER_SEC; + generator.writeNumber(millis); yield null; } - case TIMEMICRO, TIMESTAMPMICRO -> { - var tsVector = (TimeStampVector) vector; - generator.writeNumber(tsVector.get(position) / 1000); + case TIMESTAMPMICROTZ -> { + var tsVector = (TimeStampMicroTZVector) vector; + long nanos = tsVector.get(position) * NANOS_PER_MICRO; + nanos -= getUTCOffsetSeconds(nanos / NANOS_PER_MILLI, tsVector.getTimeZone()) * NANOS_PER_SEC; + generator.writeNumber(nanos); yield null; } - case TIMENANO, TIMESTAMPNANO -> { - var tsVector = (TimeStampVector) vector; - generator.writeNumber(tsVector.get(position) / 1_000_000); + case TIMESTAMPNANOTZ -> { + var tsVector = (TimeStampNanoTZVector) vector; + long nanos = tsVector.get(position); + nanos -= getUTCOffsetSeconds(nanos / NANOS_PER_SEC, tsVector.getTimeZone()) * NANOS_PER_SEC; + generator.writeNumber(nanos); yield null; } @@ -248,8 +357,7 @@ public static void writeValue(ValueVector vector, int position, Map - throw new ArrowFormatException( + LARGELISTVIEW, EXTENSIONTYPE, RUNENDENCODED -> throw new ArrowFormatException( "Arrow type [" + vector.getMinorType() + "] not supported for field [" + vector.getName() + "]" ); }; diff --git a/modules/arrow/src/test/java/org/elasticsearch/arrow/bulk/ArrowBulkIncrementalParserTests.java b/modules/arrow/src/test/java/org/elasticsearch/arrow/bulk/ArrowBulkIncrementalParserTests.java index 0e95766f6e2c2..eb49feaad48b2 100644 --- a/modules/arrow/src/test/java/org/elasticsearch/arrow/bulk/ArrowBulkIncrementalParserTests.java +++ b/modules/arrow/src/test/java/org/elasticsearch/arrow/bulk/ArrowBulkIncrementalParserTests.java @@ -93,7 +93,6 @@ private void checkBatchingAndChunking(int batchCount, int rowCount, boolean incr try (var parser = createParser("test", operations)) { parse(parser, payload, incremental); } - ; assertEquals(batchCount * rowCount, operations.size()); diff --git a/modules/arrow/src/test/java/org/elasticsearch/arrow/xcontent/ArrowToXContentTests.java b/modules/arrow/src/test/java/org/elasticsearch/arrow/xcontent/ArrowToXContentTests.java index 70e504ab232bf..11c5792c1e27e 100644 --- a/modules/arrow/src/test/java/org/elasticsearch/arrow/xcontent/ArrowToXContentTests.java +++ b/modules/arrow/src/test/java/org/elasticsearch/arrow/xcontent/ArrowToXContentTests.java @@ -9,35 +9,117 @@ package org.elasticsearch.arrow.xcontent; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.vector.BitVector; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.FixedSizeBinaryVector; +import org.apache.arrow.vector.Float4Vector; import org.apache.arrow.vector.IntVector; import org.apache.arrow.vector.NullVector; -import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.TimeMicroVector; +import org.apache.arrow.vector.TimeMilliVector; +import org.apache.arrow.vector.TimeNanoVector; +import org.apache.arrow.vector.TimeSecVector; +import org.apache.arrow.vector.TimeStampMicroTZVector; +import org.apache.arrow.vector.TimeStampMicroVector; +import org.apache.arrow.vector.TimeStampMilliTZVector; +import org.apache.arrow.vector.TimeStampMilliVector; +import org.apache.arrow.vector.TimeStampNanoTZVector; +import org.apache.arrow.vector.TimeStampNanoVector; +import org.apache.arrow.vector.TimeStampSecTZVector; +import org.apache.arrow.vector.TimeStampSecVector; +import org.apache.arrow.vector.VarBinaryVector; import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.complex.ListVector; import org.apache.arrow.vector.complex.MapVector; +import org.apache.arrow.vector.ipc.ArrowStreamReader; +import org.apache.arrow.vector.ipc.ArrowStreamWriter; import org.elasticsearch.libs.arrow.Arrow; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xcontent.XContentType; +import org.junit.After; +import org.junit.Before; +import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.time.ZoneId; +import java.util.Base64; +import java.util.List; public class ArrowToXContentTests extends ESTestCase { - private static void checkPosition(ValueVector vector, int position, String json) throws IOException { - var out = new ByteArrayOutputStream(); - try (var generator = XContentType.JSON.xContent().createGenerator(out)) { - generator.writeStartObject(); - ArrowToXContent.writeField(vector, position, null, generator); - generator.writeEndObject(); + private BufferAllocator allocator; + + @Before + public void init() { + this.allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); + } + + @After + public void close() { + this.allocator.close(); + } + + private void checkPosition(FieldVector vector, int position, String json) throws IOException { + var arrowToXContent = new ArrowToXContent(); + + var root = new VectorSchemaRoot(List.of(vector)); + // We don't close `root` as it would close `vector` which is owned by the caller. + // This allows checkPosition() to be called several times with the same vector. + + // Roundtrip the vector through its binary representation + var arrowOut = new ByteArrayOutputStream(); + try (var writer = new ArrowStreamWriter(root, null, arrowOut)) { + writer.writeBatch(); + } + + try (var reader = new ArrowStreamReader(new ByteArrayInputStream(arrowOut.toByteArray()), allocator)) { + reader.loadNextBatch(); + var newVector = reader.getVectorSchemaRoot().getVector(0); + + var jsonOut = new ByteArrayOutputStream(); + try (var generator = XContentType.JSON.xContent().createGenerator(jsonOut)) { + generator.writeStartObject(); + arrowToXContent.writeField(newVector, position, null, generator); + generator.writeEndObject(); + } + + assertEquals(json, jsonOut.toString(StandardCharsets.UTF_8)); + } + } + + // Tests below are in the same order as ArrowToXContent.writeValue() + // + // Note: dictionary encoding is tested in ArrowBulkIncrementalParserTests as + // dictionaries are attached to the StreamReader and need more than checkPosition() above. + + public void testNullValue() throws Exception { + try (var vector = new IntVector("intField", allocator)) { + vector.allocateNew(1); + vector.setNull(0); + vector.setValueCount(1); + + checkPosition(vector, 0, "{\"intField\":null}"); } + } - assertEquals(json, out.toString(StandardCharsets.UTF_8)); + public void testBoolean() throws IOException { + + try (var vector = new BitVector("bitField", allocator)) { + vector.allocateNew(1); + vector.set(0, 1); // 0 is false, other values are true + vector.setValueCount(1); + + checkPosition(vector, 0, "{\"bitField\":true}"); + } } - public void testWriteField() throws IOException { + public void testInteger() throws IOException { - try (var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); IntVector vector = new IntVector("intField", allocator);) { + try (var vector = new IntVector("intField", allocator)) { vector.allocateNew(1); vector.set(0, 123); vector.setValueCount(1); @@ -46,11 +128,20 @@ public void testWriteField() throws IOException { } } - public void testWriteVarChar() throws Exception { - try ( - var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); - VarCharVector vector = new VarCharVector("stringField", allocator); - ) { + public void testFloat() throws IOException { + float value = 1.25f; // roundtrips through string. + + try (var vector = new Float4Vector("floatField", allocator)) { + vector.allocateNew(1); + vector.set(0, value); + vector.setValueCount(1); + + checkPosition(vector, 0, "{\"floatField\":" + value + "}"); + } + } + + public void testVarChar() throws Exception { + try (var vector = new VarCharVector("stringField", allocator)) { vector.allocateNew(); vector.set(0, "test".getBytes(StandardCharsets.UTF_8)); vector.setValueCount(1); @@ -59,11 +150,156 @@ public void testWriteVarChar() throws Exception { } } - public void testWriteMap() throws Exception { - try ( - var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); - MapVector vector = MapVector.empty("mapField", allocator, false); - ) { + public void testBinary() throws Exception { + var value = "test".getBytes(StandardCharsets.UTF_8); + var expected = Base64.getEncoder().encodeToString(value); + + try (var vector = new VarBinaryVector("bytesField", allocator)) { + vector.allocateNew(); + vector.set(0, value); + vector.setValueCount(1); + + checkPosition(vector, 0, "{\"bytesField\":\"" + expected + "\"}"); + } + + try (var vector = new FixedSizeBinaryVector("bytesField", allocator, value.length)) { + vector.allocateNew(); + vector.set(0, value); + vector.setValueCount(1); + + checkPosition(vector, 0, "{\"bytesField\":\"" + expected + "\"}"); + } + } + + public void testList() throws Exception { + try (var vector = ListVector.empty("listField", allocator)) { + var w = vector.getWriter(); + + w.startList(); + w.writeInt(1); + w.writeInt(2); + w.endList(); + + w.startList(); + w.writeInt(3); + w.writeInt(4); + w.writeInt(5); + w.endList(); + w.setValueCount(w.getPosition()); + + checkPosition(vector, 0, "{\"listField\":[1,2]}"); + checkPosition(vector, 1, "{\"listField\":[3,4,5]}"); + } + } + + public void testTime() throws Exception { + var millis = randomIntBetween(0, 24 * 60 * 60 * 1000 - 1); + + try (var vector = new TimeSecVector("intField", allocator)) { + vector.allocateNew(1); + vector.set(0, millis / 1000); + vector.setValueCount(1); + checkPosition(vector, 0, "{\"intField\":" + (millis / 1000 * 1000) + "}"); + } + + try (var vector = new TimeMilliVector("intField", allocator)) { + vector.allocateNew(1); + vector.set(0, millis); + vector.setValueCount(1); + checkPosition(vector, 0, "{\"intField\":" + millis + "}"); + } + + var nanos = randomLongBetween(0, 24 * 60 * 60 * 1000_000_000L - 1); + + try (var vector = new TimeMicroVector("intField", allocator)) { + vector.allocateNew(1); + vector.set(0, nanos / 1000L); + vector.setValueCount(1); + checkPosition(vector, 0, "{\"intField\":" + (nanos / 1000L * 1000L) + "}"); + } + + try (var vector = new TimeNanoVector("intField", allocator)) { + vector.allocateNew(1); + vector.set(0, nanos); + vector.setValueCount(1); + checkPosition(vector, 0, "{\"intField\":" + nanos + "}"); + } + } + + public void testTimeStamp() throws Exception { + var millis = 1744304614884L; // Thu Apr 10 19:03:34 CEST 2025 + + try (var vector = new TimeStampSecVector("intField", allocator)) { + vector.allocateNew(1); + vector.set(0, millis / 1000L); + vector.setValueCount(1); + checkPosition(vector, 0, "{\"intField\":" + (millis / 1000L * 1000L) + "}"); + } + + try (var vector = new TimeStampMilliVector("intField", allocator)) { + vector.allocateNew(1); + vector.set(0, millis); + vector.setValueCount(1); + checkPosition(vector, 0, "{\"intField\":" + millis + "}"); + } + + var nanos = millis * 1_000_000L + 123_456L; + + try (var vector = new TimeStampMicroVector("intField", allocator)) { + vector.allocateNew(1); + vector.set(0, nanos / 1000L); + vector.setValueCount(1); + checkPosition(vector, 0, "{\"intField\":" + (nanos / 1000L * 1000L) + "}"); + } + + try (var vector = new TimeStampNanoVector("intField", allocator)) { + vector.allocateNew(1); + vector.set(0, nanos); + vector.setValueCount(1); + checkPosition(vector, 0, "{\"intField\":" + nanos + "}"); + } + } + + public void testTimeStampTZ() throws Exception { + var tz = ZoneId.of("UTC+1"); + + var millis = 1744304614884L; // Thu Apr 10 19:03:34 CEST 2025 + var wallMillis = millis + 3_600_000L; // UTC+1 is 1 hour ahead of UTC + + try (var vector = new TimeStampSecTZVector("intField", allocator, tz.getId())) { + vector.allocateNew(1); + vector.set(0, wallMillis / 1000L); + vector.setValueCount(1); + checkPosition(vector, 0, "{\"intField\":" + (millis / 1000L * 1000L) + "}"); + } + + try (var vector = new TimeStampMilliTZVector("intField", allocator, tz.getId())) { + vector.allocateNew(1); + vector.set(0, wallMillis); + vector.setValueCount(1); + checkPosition(vector, 0, "{\"intField\":" + millis + "}"); + } + + var nanos = millis * 1_000_000L + 123_456L; + var wallNanos = wallMillis * 1_000_000L + 123_456L; + + try (var vector = new TimeStampMicroTZVector("intField", allocator, tz.getId())) { + vector.allocateNew(1); + vector.set(0, wallNanos / 1000L); + vector.setValueCount(1); + checkPosition(vector, 0, "{\"intField\":" + (nanos / 1000L * 1000L) + "}"); + } + + try (var vector = new TimeStampNanoTZVector("intField", allocator, tz.getId())) { + vector.allocateNew(1); + vector.set(0, wallNanos); + vector.setValueCount(1); + checkPosition(vector, 0, "{\"intField\":" + nanos + "}"); + } + } + + public void testMap() throws Exception { + try (var vector = MapVector.empty("mapField", allocator, false)) { var w = vector.getWriter(); w.startMap(); @@ -72,24 +308,20 @@ public void testWriteMap() throws Exception { w.value().integer().writeInt(42); w.endEntry(); w.endMap(); + w.setValueCount(w.getPosition()); checkPosition(vector, 0, "{\"mapField\":{\"key1\":42}}"); } } - public void testWriteNullValue() throws Exception { - try (var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); IntVector vector = new IntVector("intField", allocator);) { - vector.allocateNew(1); - vector.setNull(0); - vector.setValueCount(1); - - checkPosition(vector, 0, "{\"intField\":null}"); - } - } + // TODO: struct (already exercised in ArrowBulkIncrementalParserTests) + // TODO: dense union + // TODO: union (already exercised in ArrowBulkIncrementalParserTests) - public void testWriteNullVector() throws Exception { + public void testNullVector() throws Exception { try (NullVector vector = new NullVector("nullField", 1);) { checkPosition(vector, 0, "{\"nullField\":null}"); } } + } From 98e2e7e9bd23aeeac19129ec670bbc52abf6b298 Mon Sep 17 00:00:00 2001 From: Sylvain Wallez Date: Sat, 12 Apr 2025 11:32:36 +0200 Subject: [PATCH 19/19] Return an Arrow response --- .../arrow/bulk/ArrowBulkActionIT.java | 30 +++-- .../arrow/bulk/ArrowBulkAction.java | 107 +++++++++++++++++- .../rest/action/document/RestBulkAction.java | 7 +- 3 files changed, 131 insertions(+), 13 deletions(-) diff --git a/modules/arrow/src/internalClusterTest/java/org/elasticsearch/arrow/bulk/ArrowBulkActionIT.java b/modules/arrow/src/internalClusterTest/java/org/elasticsearch/arrow/bulk/ArrowBulkActionIT.java index 04fb4d9679375..6dfe20595e939 100644 --- a/modules/arrow/src/internalClusterTest/java/org/elasticsearch/arrow/bulk/ArrowBulkActionIT.java +++ b/modules/arrow/src/internalClusterTest/java/org/elasticsearch/arrow/bulk/ArrowBulkActionIT.java @@ -12,6 +12,7 @@ import org.apache.arrow.vector.IntVector; import org.apache.arrow.vector.VarCharVector; import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.ipc.ArrowStreamReader; import org.apache.arrow.vector.ipc.ArrowStreamWriter; import org.apache.arrow.vector.types.pojo.ArrowType; import org.apache.arrow.vector.types.pojo.Field; @@ -78,8 +79,8 @@ public void testBulk() throws Exception { Field strField = new Field("strings", FieldType.nullable(new ArrowType.Utf8()), null); Schema schema = new Schema(List.of(intField, strField)); - int batchCount = randomInt(10); - int rowCount = randomInt(10); + int batchCount = randomIntBetween(1, 10); + int rowCount = randomIntBetween(1, 10); byte[] payload; @@ -91,12 +92,10 @@ public void testBulk() throws Exception { try (ArrowStreamWriter writer = new ArrowStreamWriter(root, null, baos)) { for (int batch = 0; batch < batchCount; batch++) { - intVector.allocateNew(rowCount); - stringVector.allocateNew(rowCount); for (int row = 0; row < rowCount; row++) { int globalRow = row + batch * rowCount; - intVector.set(row, globalRow); - stringVector.set(row, new Text("row" + globalRow)); + intVector.setSafe(row, globalRow); + stringVector.setSafe(row, new Text("row" + globalRow)); } root.setRowCount(rowCount); writer.writeBatch(); @@ -109,15 +108,24 @@ public void testBulk() throws Exception { // Bulk insert the arrow stream var request = new Request("POST", "/_arrow/" + index + "/_bulk"); request.addParameter("refresh", "wait_for"); + request.addParameter("error_trace", "true"); + request.setOptions(request.getOptions().toBuilder().addHeader("Content-type", "application/vnd.apache.arrow.stream")); request.setEntity(new ByteArrayEntity(payload, ContentType.create(Arrow.MEDIA_TYPE))); var response = restClient.performRequest(request); - var result = XContentType.JSON.xContent() - .createParser(XContentParserConfiguration.EMPTY, response.getEntity().getContent()) - .map(); - assertEquals(Boolean.FALSE, result.get("errors")); - assertEquals(batchCount * rowCount, ((List) result.get("items")).size()); + // Response is an Arrow stream with empty vectors, indicating success + assertEquals(Arrow.MEDIA_TYPE, response.getHeader("Content-Type")); + try ( + var allocator = Arrow.newChildAllocator("test", 0, Long.MAX_VALUE); + var reader = new ArrowStreamReader(response.getEntity().getContent(), allocator); + ) { + reader.loadNextBatch(); + var root = reader.getVectorSchemaRoot(); + var itemNoVector = root.getVector(ArrowBulkAction.ERR_ITEM_NO); + assertNotNull(itemNoVector); + assertEquals(0, itemNoVector.getValueCount()); + } } { diff --git a/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkAction.java b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkAction.java index d74591d3a168d..34c94c6e1703f 100644 --- a/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkAction.java +++ b/modules/arrow/src/main/java/org/elasticsearch/arrow/bulk/ArrowBulkAction.java @@ -9,6 +9,16 @@ package org.elasticsearch.arrow.bulk; +import org.apache.arrow.vector.UInt2Vector; +import org.apache.arrow.vector.UInt4Vector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.ipc.ArrowStreamWriter; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.bulk.BulkShardRequest; import org.elasticsearch.action.bulk.IncrementalBulkService; import org.elasticsearch.client.internal.Client; @@ -16,11 +26,16 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.IndexingPressure; +import org.elasticsearch.libs.arrow.Arrow; import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.RestResponse; +import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.action.document.RestBulkAction; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.List; import static org.elasticsearch.rest.RestRequest.Method.POST; @@ -28,6 +43,15 @@ public class ArrowBulkAction extends BaseRestHandler { + public static final String ID = "_id"; + public static final String INDEX = "_index"; + public static final String ACTION = "_bulk_action"; + + public static final String ERR_ITEM_NO = "item_no"; + public static final String ERR_STATUS = "status"; + public static final String ERR_TYPE = "type"; + public static final String ERR_REASON = "reason"; + private final IncrementalBulkService bulkHandler; public ArrowBulkAction(Client client, Settings settings) { @@ -63,6 +87,87 @@ protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient cli request, () -> bulkHandler.newBulkRequest(waitForActiveShards, timeout, refresh), new ArrowBulkRequestParser(request) - ); + ) { + @Override + protected ActionListener createResponseListener(RestChannel channel) { + // Return an Arrow response if Accept is missing or is the Arrow media type. + // The Arrow response only contains failures, and will be an empty table if there are no failures. + // Global (request-level) failures are always returned as JSON. + + var parent = super.createResponseListener(channel); + var accept = request.header("Accept"); + if (accept == null || accept.equals(Arrow.MEDIA_TYPE)) { + return new ArrowResponseListener(channel, parent); + } else { + return parent; + } + } + }; + } + + private record ArrowResponseListener(RestChannel channel, ActionListener parent) implements ActionListener { + + @Override + public void onResponse(BulkResponse bulkItemResponses) { + // FIXME: we can be more efficient and stream the response, like we do in ESQL's ArrowResponse + var output = new BytesReferenceOutputStream(); + try ( + var allocator = Arrow.newChildAllocator("bulk_response", 0, 10_000_000L); + var itemNoVector = new UInt4Vector(ERR_ITEM_NO, allocator); + var indexVector = new VarCharVector(INDEX, allocator); // Could be dictionary-encoded + var idVector = new VarCharVector(ID, allocator); + var statusVector = new UInt2Vector(ERR_STATUS, allocator); + var typeVector = new VarCharVector(ERR_TYPE, allocator); // Could be dictionary-encoded + var reasonVector = new VarCharVector(ERR_REASON, allocator); + + var root = new VectorSchemaRoot(List.of(itemNoVector, indexVector, idVector, statusVector, typeVector, reasonVector)); + var writer = new ArrowStreamWriter(root, null, output); + ) { + int failureCount = 0; + var items = bulkItemResponses.getItems(); + for (int itemNo = 0; itemNo < items.length; itemNo++) { + var item = items[itemNo]; + if (item.isFailed()) { + BulkItemResponse.Failure failure = item.getFailure(); + + itemNoVector.setSafe(failureCount, itemNo); + addValue(indexVector, failureCount, failure.getIndex()); + addValue(idVector, failureCount, failure.getId()); + statusVector.setSafe(failureCount, failure.getStatus().getStatus()); + + Throwable cause = ExceptionsHelper.unwrapCause(failure.getCause()); + addValue(typeVector, failureCount, cause == null ? null : ElasticsearchException.getExceptionName(cause)); + addValue(reasonVector, failureCount, cause == null ? null : cause.getMessage()); + + failureCount++; + } + } + + for (var vec : root.getFieldVectors()) { + vec.setValueCount(failureCount); + } + root.setRowCount(failureCount); + + writer.writeBatch(); + } catch (IOException e) { + this.onFailure(e); + } + var response = new RestResponse(RestStatus.OK, Arrow.MEDIA_TYPE, output.asBytesReference()); + channel.sendResponse(response); + } + + private void addValue(VarCharVector vector, int position, String value) { + if (value == null) { + vector.setNull(position); + } else { + vector.setSafe(position, value.getBytes(StandardCharsets.UTF_8)); + } + } + + @Override + public void onFailure(Exception e) { + // Output the failure as JSON + parent.onFailure(e); + } } } diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java index cd382090b4f3d..ccfc900c72262 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java @@ -15,6 +15,7 @@ import org.elasticsearch.action.bulk.AbstractBulkRequestParser; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkRequestParser; +import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.bulk.BulkShardRequest; import org.elasticsearch.action.bulk.IncrementalBulkService; import org.elasticsearch.action.support.ActiveShardCount; @@ -199,6 +200,10 @@ public void accept(RestChannel restChannel) { request.contentStream().next(); } + protected ActionListener createResponseListener(RestChannel channel) { + return new RestRefCountedChunkedToXContentListener<>(channel); + } + @Override public void handleChunk(RestChannel channel, ReleasableBytesReference chunk, boolean isLast) { assert handler != null; @@ -244,7 +249,7 @@ public void handleChunk(RestChannel channel, ReleasableBytesReference chunk, boo assert channel != null; ArrayList> toPass = new ArrayList<>(items); items.clear(); - handler.lastItems(toPass, () -> Releasables.close(releasables), new RestRefCountedChunkedToXContentListener<>(channel)); + handler.lastItems(toPass, () -> Releasables.close(releasables), createResponseListener(channel)); } } else if (items.isEmpty() == false) { ArrayList> toPass = new ArrayList<>(items);