Teach SQL to parse Elasticsearch's standard error responses (elastic/x-pack-elasticsearch#2764)
This teaches SQL to parse Elasticsearch's standard error responses but doesn't change SQL to general Elasticsearch's standard error responses in all cases. That can come in a followup. We do this parsing with jackson-core, the same dependency Elasticsearch uses for parsing json. We shade jackson-core in the JDBC driver so that users don't have to worry about dependency clashes. We do not do so in the CLI because it is a standalone application. We get a few "bonus" changes along the way: 1. We save a copy operation. Before this change responses were spooled into memory and then parsed. After this change they are parsed directly from the response stream. 2. We had a few classes entirely to support the spooling operation that we no longer need: `BytesArray`, `FastByteArrayInputStream`, and `BasicByteArrayOutputStream`. 3. SQL's `Version` was incorrectly parsing the version from the jar manifest. We didn't notice because the test was rigged to return `UNKNOWN` because we *were* running the test from the compiled classes directory instead of the jar. As part of shading jackson we moved running the tests to running against the shaded jar. Now we can actually assert that we parse the version correctly. It turns out we weren't. So I fixed it. Original commit: elastic/x-pack-elasticsearch@2e8f397bf4
This commit is contained in:
parent
94d0a2d1ee
commit
89e80e0cba
|
@ -8,7 +8,7 @@ dependencies {
|
|||
compile "org.elasticsearch.test:framework:${versions.elasticsearch}"
|
||||
|
||||
// JDBC testing dependencies
|
||||
compile(project(':x-pack-elasticsearch:sql:jdbc')) {
|
||||
compile(project(path: ':x-pack-elasticsearch:sql:jdbc', configuration: 'shadow')) {
|
||||
if (false == isEclipse && false == isIdea) {
|
||||
/* Skip the transitive dependencies of the server when outside
|
||||
* of an IDE because outside of an IDE we use the jdbc jar
|
||||
|
|
|
@ -12,12 +12,14 @@ dependencies {
|
|||
|
||||
runtime "org.fusesource.jansi:jansi:1.16"
|
||||
runtime "org.elasticsearch:jna:4.4.0-1"
|
||||
runtime "com.fasterxml.jackson.core:jackson-core:${versions.jackson}"
|
||||
}
|
||||
|
||||
dependencyLicenses {
|
||||
mapping from: /cli-proto.*/, to: 'elasticsearch'
|
||||
mapping from: /shared-client.*/, to: 'elasticsearch'
|
||||
mapping from: /shared-proto.*/, to: 'elasticsearch'
|
||||
mapping from: /jackson-.*/, to: 'jackson'
|
||||
ignoreSha 'cli-proto'
|
||||
ignoreSha 'shared-client'
|
||||
ignoreSha 'shared-proto'
|
||||
|
|
|
@ -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
|
|
@ -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.
|
|
@ -0,0 +1 @@
|
|||
eb21a035c66ad307e66ec8fce37f5d50fd62d039
|
|
@ -7,9 +7,8 @@ package org.elasticsearch.xpack.sql.cli;
|
|||
|
||||
import org.elasticsearch.xpack.sql.cli.net.protocol.Proto.ResponseType;
|
||||
import org.elasticsearch.xpack.sql.cli.net.protocol.QueryResponse;
|
||||
import org.elasticsearch.xpack.sql.client.shared.IOUtils;
|
||||
import org.elasticsearch.xpack.sql.client.shared.StringUtils;
|
||||
import org.elasticsearch.xpack.sql.client.shared.SuppressForbidden;
|
||||
import org.elasticsearch.xpack.sql.client.shared.StringUtils;
|
||||
import org.elasticsearch.xpack.sql.protocol.shared.AbstractQueryInitRequest;
|
||||
import org.elasticsearch.xpack.sql.protocol.shared.Response;
|
||||
import org.jline.reader.EndOfFileException;
|
||||
|
@ -25,9 +24,11 @@ import org.jline.utils.InfoCmp.Capability;
|
|||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.InputStreamReader;
|
||||
import java.io.PrintWriter;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Locale;
|
||||
import java.util.Properties;
|
||||
import java.util.logging.LogManager;
|
||||
|
@ -92,7 +93,7 @@ public class Cli {
|
|||
}
|
||||
|
||||
boolean debug = StringUtils.parseBoolean(System.getProperty("cli.debug", "false"));
|
||||
Cli console = new Cli(debug, new CliConfiguration(hostAndPort + "/_sql/cli", properties), term);
|
||||
Cli console = new Cli(debug, new CliConfiguration(hostAndPort + "/_sql/cli?error_trace", properties), term);
|
||||
console.run();
|
||||
} catch (FatalException e) {
|
||||
term.writer().println(e.getMessage());
|
||||
|
@ -119,7 +120,7 @@ public class Cli {
|
|||
.terminal(term)
|
||||
.completer(Completers.INSTANCE)
|
||||
.build();
|
||||
|
||||
|
||||
String DEFAULT_PROMPT = new AttributedString("sql> ", DEFAULT.foreground(YELLOW)).toAnsi(term);
|
||||
String MULTI_LINE_PROMPT = new AttributedString(" | ", DEFAULT.foreground(YELLOW)).toAnsi(term);
|
||||
|
||||
|
@ -187,7 +188,7 @@ public class Cli {
|
|||
|
||||
/**
|
||||
* Handle an exception while communication with the server. Extracted
|
||||
* into a method so that tests can bubble the failure.
|
||||
* into a method so that tests can bubble the failure.
|
||||
*/
|
||||
protected void handleExceptionWhileCommunicatingWithServer(RuntimeException e) {
|
||||
AttributedStringBuilder asb = new AttributedStringBuilder();
|
||||
|
@ -200,20 +201,22 @@ public class Cli {
|
|||
}
|
||||
}
|
||||
|
||||
private static String logo() {
|
||||
try (InputStream io = Cli.class.getResourceAsStream("/logo.txt")) {
|
||||
if (io == null) {
|
||||
private void printLogo() {
|
||||
term.puts(Capability.clear_screen);
|
||||
try (InputStream in = Cli.class.getResourceAsStream("/logo.txt")) {
|
||||
if (in == null) {
|
||||
throw new FatalException("Could not find logo!");
|
||||
}
|
||||
return IOUtils.asBytes(io).toString();
|
||||
try (BufferedReader reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8))) {
|
||||
String line;
|
||||
while ((line = reader.readLine()) != null) {
|
||||
term.writer().println(line);
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new FatalException("Could not load logo!", e);
|
||||
}
|
||||
}
|
||||
|
||||
private void printLogo() {
|
||||
term.puts(Capability.clear_screen);
|
||||
term.writer().println(logo());
|
||||
term.writer().println();
|
||||
}
|
||||
|
||||
|
@ -273,7 +276,7 @@ public class Cli {
|
|||
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
private boolean isServerInfo(String line) {
|
||||
line = line.toLowerCase(Locale.ROOT);
|
||||
return line.equals("info");
|
||||
|
@ -322,4 +325,4 @@ public class Cli {
|
|||
System.err.println(message);
|
||||
System.exit(code);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -5,29 +5,30 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.sql.cli;
|
||||
|
||||
import org.elasticsearch.xpack.sql.cli.net.protocol.ErrorResponse;
|
||||
import org.elasticsearch.xpack.sql.cli.net.protocol.ExceptionResponse;
|
||||
import org.elasticsearch.xpack.sql.cli.net.protocol.InfoRequest;
|
||||
import org.elasticsearch.xpack.sql.cli.net.protocol.InfoResponse;
|
||||
import org.elasticsearch.xpack.sql.cli.net.protocol.Proto;
|
||||
import org.elasticsearch.xpack.sql.cli.net.protocol.Proto.RequestType;
|
||||
import org.elasticsearch.xpack.sql.cli.net.protocol.QueryInitRequest;
|
||||
import org.elasticsearch.xpack.sql.cli.net.protocol.QueryPageRequest;
|
||||
import org.elasticsearch.xpack.sql.client.shared.Bytes;
|
||||
import org.elasticsearch.xpack.sql.protocol.shared.AbstractProto.SqlExceptionType;
|
||||
import org.elasticsearch.xpack.sql.client.shared.JreHttpUrlConnection;
|
||||
import org.elasticsearch.xpack.sql.protocol.shared.Request;
|
||||
import org.elasticsearch.xpack.sql.protocol.shared.Response;
|
||||
import org.elasticsearch.xpack.sql.protocol.shared.TimeoutInfo;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.IOException;
|
||||
import java.security.AccessController;
|
||||
import java.security.PrivilegedAction;
|
||||
import java.time.Instant;
|
||||
import java.util.TimeZone;
|
||||
|
||||
public class CliHttpClient {
|
||||
private final HttpClient http;
|
||||
private final CliConfiguration cfg;
|
||||
|
||||
public CliHttpClient(CliConfiguration cfg) {
|
||||
this.cfg = cfg;
|
||||
this.http = new HttpClient(cfg);
|
||||
}
|
||||
|
||||
public InfoResponse serverInfo() {
|
||||
|
@ -51,14 +52,23 @@ public class CliHttpClient {
|
|||
long clientTime = Instant.now().toEpochMilli();
|
||||
return new TimeoutInfo(clientTime, cfg.queryTimeout(), cfg.pageTimeout());
|
||||
}
|
||||
|
||||
private Response sendRequest(Request request) {
|
||||
Bytes ba = http.post(out -> Proto.INSTANCE.writeRequest(request, out));
|
||||
try (DataInputStream in = new DataInputStream(new ByteArrayInputStream(ba.bytes(), 0, ba.size()))) {
|
||||
return Proto.INSTANCE.readResponse(request, in);
|
||||
} catch (IOException ex) {
|
||||
throw new CliException(ex, "Cannot read response");
|
||||
}
|
||||
return AccessController.doPrivileged((PrivilegedAction<Response>) () ->
|
||||
JreHttpUrlConnection.http(cfg.asUrl(), cfg, con ->
|
||||
con.post(
|
||||
out -> Proto.INSTANCE.writeRequest(request, out),
|
||||
in -> Proto.INSTANCE.readResponse(request, in),
|
||||
(status, failure) -> {
|
||||
if (status >= 500) {
|
||||
return new ErrorResponse((RequestType) request.requestType(), failure.reason(),
|
||||
failure.type(), failure.remoteTrace());
|
||||
}
|
||||
return new ExceptionResponse((RequestType) request.requestType(), failure.reason(),
|
||||
failure.type(), SqlExceptionType.fromRemoteFailureType(failure.type()));
|
||||
}
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -1,37 +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;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.sql.cli;
|
||||
|
||||
import org.elasticsearch.xpack.sql.client.shared.Bytes;
|
||||
import org.elasticsearch.xpack.sql.client.shared.CheckedConsumer;
|
||||
import org.elasticsearch.xpack.sql.client.shared.ClientException;
|
||||
import org.elasticsearch.xpack.sql.client.shared.JreHttpUrlConnection;
|
||||
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.security.AccessController;
|
||||
import java.security.PrivilegedAction;
|
||||
|
||||
class HttpClient {
|
||||
|
||||
private final CliConfiguration cfg;
|
||||
|
||||
HttpClient(CliConfiguration cfg) {
|
||||
this.cfg = cfg;
|
||||
}
|
||||
|
||||
Bytes post(CheckedConsumer<DataOutput, IOException> os) {
|
||||
try {
|
||||
return AccessController.doPrivileged((PrivilegedAction<Bytes>) () -> {
|
||||
return JreHttpUrlConnection.http(cfg.asUrl(), cfg, con -> {
|
||||
return con.post(os);
|
||||
});
|
||||
});
|
||||
} catch (ClientException ex) {
|
||||
throw new RuntimeException("Transport failure", ex);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,3 +1,7 @@
|
|||
plugins {
|
||||
id 'com.github.johnrengelman.shadow' version '2.0.1'
|
||||
}
|
||||
|
||||
import org.elasticsearch.gradle.test.RunTask
|
||||
|
||||
description = 'JDBC driver for Elasticsearch'
|
||||
|
@ -11,23 +15,50 @@ dependencies {
|
|||
compile project(':x-pack-elasticsearch:sql:shared-client')
|
||||
compile project(':x-pack-elasticsearch:sql:jdbc-proto')
|
||||
compile project(':x-pack-elasticsearch:sql:shared-proto')
|
||||
/* We want to limit these dependencies so do not add anything to this list
|
||||
* without serious consideration, and probably shading. */
|
||||
runtime "com.fasterxml.jackson.core:jackson-core:${versions.jackson}"
|
||||
/* We want to limit these dependencies so we don't have a huge jar.
|
||||
* Since we shadow these dependencies we don't have to be super careful
|
||||
* but we have to be *somewhat* careful because things like commons logging
|
||||
* don't shadow properly. */
|
||||
}
|
||||
|
||||
dependencyLicenses {
|
||||
mapping from: /jdbc-proto.*/, to: 'elasticsearch'
|
||||
mapping from: /shared-client.*/, to: 'elasticsearch'
|
||||
mapping from: /shared-proto.*/, to: 'elasticsearch'
|
||||
mapping from: /jackson-.*/, to: 'jackson'
|
||||
ignoreSha 'jdbc-proto'
|
||||
ignoreSha 'shared-client'
|
||||
ignoreSha 'shared-proto'
|
||||
}
|
||||
|
||||
/* Disable the jar task configured by the java plugin. We're not going to
|
||||
* distribute an unshaded jar so there is no need making one. */
|
||||
jar {
|
||||
// Bundle all dependencies into the jar.
|
||||
from {
|
||||
configurations.compile.collect { it.isDirectory() ? it : zipTree(it) }
|
||||
configurations.runtime.collect { it.isDirectory() ? it : zipTree(it) }
|
||||
enabled = false
|
||||
}
|
||||
configurations.archives.artifacts.removeAll { it.archiveTask.is jar }
|
||||
|
||||
/* Move the shaded jar to the empty classifier because it is the only one
|
||||
* we're shipping. */
|
||||
shadowJar {
|
||||
classifier = null
|
||||
// We only need to relocate jackson
|
||||
relocate 'com.fasterxml.jackson', 'org.elasticsearch.xpack.sql.jdbc.shadow.jacksonp'
|
||||
manifest {
|
||||
inheritFrom jar.manifest
|
||||
}
|
||||
}
|
||||
assemble.dependsOn shadowJar
|
||||
artifacts {
|
||||
archives shadowJar
|
||||
}
|
||||
|
||||
// And for better realism let's use the shaded jar for testing
|
||||
test {
|
||||
classpath -= compileJava.outputs.files
|
||||
classpath -= configurations.compile
|
||||
classpath -= configurations.runtime
|
||||
classpath += shadowJar.outputs.files
|
||||
dependsOn shadowJar
|
||||
}
|
||||
|
|
|
@ -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
|
|
@ -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.
|
|
@ -0,0 +1 @@
|
|||
eb21a035c66ad307e66ec8fce37f5d50fd62d039
|
|
@ -11,10 +11,14 @@ import org.elasticsearch.xpack.sql.client.shared.JreHttpUrlConnection;
|
|||
import org.elasticsearch.xpack.sql.jdbc.JdbcException;
|
||||
import org.elasticsearch.xpack.sql.jdbc.JdbcSQLException;
|
||||
import org.elasticsearch.xpack.sql.jdbc.jdbc.JdbcConfiguration;
|
||||
import org.elasticsearch.xpack.sql.jdbc.util.BytesArray;
|
||||
import org.elasticsearch.xpack.sql.jdbc.net.protocol.ErrorResponse;
|
||||
import org.elasticsearch.xpack.sql.jdbc.net.protocol.ExceptionResponse;
|
||||
import org.elasticsearch.xpack.sql.jdbc.net.protocol.Proto;
|
||||
import org.elasticsearch.xpack.sql.jdbc.net.protocol.Proto.RequestType;
|
||||
import org.elasticsearch.xpack.sql.protocol.shared.AbstractProto.SqlExceptionType;
|
||||
import org.elasticsearch.xpack.sql.protocol.shared.Request;
|
||||
import org.elasticsearch.xpack.sql.protocol.shared.Response;
|
||||
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.net.MalformedURLException;
|
||||
import java.net.URL;
|
||||
import java.security.AccessController;
|
||||
|
@ -37,7 +41,7 @@ class HttpClient {
|
|||
this.url = new URL(baseUrl, "_sql/jdbc?error_trace=true");
|
||||
} catch (MalformedURLException ex) {
|
||||
throw new JdbcException(ex, "Cannot connect to JDBC endpoint [" + baseUrl.toString() + "_sql/jdbc]");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void setNetworkTimeout(long millis) {
|
||||
|
@ -61,15 +65,26 @@ class HttpClient {
|
|||
}
|
||||
}
|
||||
|
||||
BytesArray put(CheckedConsumer<DataOutput, IOException> os) throws SQLException {
|
||||
Response put(Request request) throws SQLException {
|
||||
try {
|
||||
return AccessController.doPrivileged((PrivilegedAction<BytesArray>) () -> {
|
||||
return JreHttpUrlConnection.http(url, cfg, con -> {
|
||||
return new BytesArray(con.post(os));
|
||||
});
|
||||
});
|
||||
return AccessController.doPrivileged((PrivilegedAction<Response>) () ->
|
||||
JreHttpUrlConnection.http(url, cfg, con ->
|
||||
con.post(
|
||||
out -> Proto.INSTANCE.writeRequest(request, out),
|
||||
in -> Proto.INSTANCE.readResponse(request, in),
|
||||
(status, failure) -> {
|
||||
if (status >= 500) {
|
||||
return new ErrorResponse((RequestType) request.requestType(), failure.reason(),
|
||||
failure.type(), failure.remoteTrace());
|
||||
}
|
||||
return new ExceptionResponse((RequestType) request.requestType(), failure.reason(),
|
||||
failure.type(), SqlExceptionType.fromRemoteFailureType(failure.type()));
|
||||
}
|
||||
)
|
||||
)
|
||||
);
|
||||
} catch (ClientException ex) {
|
||||
throw new JdbcSQLException(ex, "Transport failure");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,8 +23,6 @@ import org.elasticsearch.xpack.sql.jdbc.net.protocol.QueryInitRequest;
|
|||
import org.elasticsearch.xpack.sql.jdbc.net.protocol.QueryInitResponse;
|
||||
import org.elasticsearch.xpack.sql.jdbc.net.protocol.QueryPageRequest;
|
||||
import org.elasticsearch.xpack.sql.jdbc.net.protocol.QueryPageResponse;
|
||||
import org.elasticsearch.xpack.sql.jdbc.util.BytesArray;
|
||||
import org.elasticsearch.xpack.sql.jdbc.util.FastByteArrayInputStream;
|
||||
import org.elasticsearch.xpack.sql.protocol.shared.Request;
|
||||
import org.elasticsearch.xpack.sql.protocol.shared.Response;
|
||||
import org.elasticsearch.xpack.sql.protocol.shared.TimeoutInfo;
|
||||
|
@ -66,8 +64,7 @@ public class JdbcHttpClient {
|
|||
public Cursor query(String sql, RequestMeta meta) throws SQLException {
|
||||
int fetch = meta.fetchSize() > 0 ? meta.fetchSize() : conCfg.pageSize();
|
||||
QueryInitRequest request = new QueryInitRequest(sql, fetch, conCfg.timeZone(), timeout(meta));
|
||||
BytesArray ba = http.put(out -> Proto.INSTANCE.writeRequest(request, out));
|
||||
QueryInitResponse response = doIO(ba, in -> (QueryInitResponse) readResponse(request, in));
|
||||
QueryInitResponse response = (QueryInitResponse) checkResponse(http.put(request));
|
||||
return new DefaultCursor(this, response.cursor(), (Page) response.data, meta);
|
||||
}
|
||||
|
||||
|
@ -77,8 +74,7 @@ public class JdbcHttpClient {
|
|||
*/
|
||||
public byte[] nextPage(byte[] cursor, Page page, RequestMeta meta) throws SQLException {
|
||||
QueryPageRequest request = new QueryPageRequest(cursor, timeout(meta), page);
|
||||
BytesArray ba = http.put(out -> Proto.INSTANCE.writeRequest(request, out));
|
||||
return doIO(ba, in -> ((QueryPageResponse) readResponse(request, in)).cursor());
|
||||
return ((QueryPageResponse) checkResponse(http.put(request))).cursor();
|
||||
}
|
||||
|
||||
public InfoResponse serverInfo() throws SQLException {
|
||||
|
@ -90,20 +86,17 @@ public class JdbcHttpClient {
|
|||
|
||||
private InfoResponse fetchServerInfo() throws SQLException {
|
||||
InfoRequest request = new InfoRequest();
|
||||
BytesArray ba = http.put(out -> Proto.INSTANCE.writeRequest(request, out));
|
||||
return doIO(ba, in -> (InfoResponse) readResponse(request, in));
|
||||
return (InfoResponse) checkResponse(http.put(request));
|
||||
}
|
||||
|
||||
public List<String> metaInfoTables(String pattern) throws SQLException {
|
||||
MetaTableRequest request = new MetaTableRequest(pattern);
|
||||
BytesArray ba = http.put(out -> Proto.INSTANCE.writeRequest(request, out));
|
||||
return doIO(ba, in -> ((MetaTableResponse) readResponse(request, in)).tables);
|
||||
return ((MetaTableResponse) checkResponse(http.put(request))).tables;
|
||||
}
|
||||
|
||||
public List<MetaColumnInfo> metaInfoColumns(String tablePattern, String columnPattern) throws SQLException {
|
||||
MetaColumnRequest request = new MetaColumnRequest(tablePattern, columnPattern);
|
||||
BytesArray ba = http.put(out -> Proto.INSTANCE.writeRequest(request, out));
|
||||
return doIO(ba, in -> ((MetaColumnResponse) readResponse(request, in)).columns);
|
||||
return ((MetaColumnResponse) checkResponse(http.put(request))).columns;
|
||||
}
|
||||
|
||||
public void setNetworkTimeout(long millis) {
|
||||
|
@ -114,17 +107,7 @@ public class JdbcHttpClient {
|
|||
return http.getNetworkTimeout();
|
||||
}
|
||||
|
||||
private static <T> T doIO(BytesArray ba, DataInputFunction<T> action) throws SQLException {
|
||||
try (DataInputStream in = new DataInputStream(new FastByteArrayInputStream(ba))) {
|
||||
return action.apply(in);
|
||||
} catch (IOException ex) {
|
||||
throw new JdbcSQLException(ex, "Cannot read response");
|
||||
}
|
||||
}
|
||||
|
||||
private static Response readResponse(Request request, DataInput in) throws IOException, SQLException {
|
||||
Response response = Proto.INSTANCE.readResponse(request, in);
|
||||
|
||||
private static Response checkResponse(Response response) throws SQLException {
|
||||
if (response.responseType() == ResponseType.EXCEPTION) {
|
||||
ExceptionResponse ex = (ExceptionResponse) response;
|
||||
throw ex.asException();
|
||||
|
|
|
@ -1,147 +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;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.sql.jdbc.util;
|
||||
|
||||
import org.elasticsearch.xpack.sql.client.shared.Bytes;
|
||||
import org.elasticsearch.xpack.sql.client.shared.StringUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.sql.SQLException;
|
||||
|
||||
public class BytesArray {
|
||||
|
||||
public static final byte[] EMPTY = new byte[0];
|
||||
|
||||
byte[] bytes = EMPTY;
|
||||
int offset = 0;
|
||||
int size = 0;
|
||||
|
||||
public BytesArray(int size) {
|
||||
this(new byte[size], 0, 0);
|
||||
}
|
||||
|
||||
public BytesArray(byte[] data) {
|
||||
this(data, 0, data.length);
|
||||
}
|
||||
|
||||
public BytesArray(byte[] data, int size) {
|
||||
this(data, 0, size);
|
||||
}
|
||||
|
||||
public BytesArray(byte[] data, int offset, int size) {
|
||||
this.bytes = data;
|
||||
this.offset = offset;
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
public BytesArray(Bytes bytes) {
|
||||
this(bytes.bytes(), 0, bytes.size());
|
||||
}
|
||||
|
||||
public byte[] bytes() {
|
||||
return bytes;
|
||||
}
|
||||
|
||||
public int offset() {
|
||||
return offset;
|
||||
}
|
||||
|
||||
public int length() {
|
||||
return size;
|
||||
}
|
||||
|
||||
public int capacity() {
|
||||
return bytes.length;
|
||||
}
|
||||
|
||||
public int available() {
|
||||
return bytes.length - size;
|
||||
}
|
||||
|
||||
public void bytes(byte[] array) {
|
||||
this.bytes = array;
|
||||
this.size = array.length;
|
||||
this.offset = 0;
|
||||
}
|
||||
|
||||
public void bytes(byte[] array, int size) {
|
||||
this.bytes = array;
|
||||
this.size = size;
|
||||
this.offset = 0;
|
||||
}
|
||||
|
||||
public void bytes(BytesArray ba) {
|
||||
this.bytes = ba.bytes;
|
||||
this.size = ba.size;
|
||||
this.offset = ba.offset;
|
||||
}
|
||||
|
||||
public void size(int size) {
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
public void offset(int offset) {
|
||||
this.offset = offset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return StringUtils.asHexString(bytes, offset, size);
|
||||
}
|
||||
|
||||
public void reset() {
|
||||
offset = 0;
|
||||
size = 0;
|
||||
}
|
||||
|
||||
public void copyTo(BytesArray to) throws SQLException {
|
||||
to.add(bytes, offset, size);
|
||||
}
|
||||
|
||||
public void add(int b) throws SQLException {
|
||||
int newcount = size + 1;
|
||||
checkSize(newcount);
|
||||
bytes[size] = (byte) b;
|
||||
size = newcount;
|
||||
}
|
||||
|
||||
public void add(byte[] b) throws SQLException {
|
||||
if (b == null || b.length == 0) {
|
||||
return;
|
||||
}
|
||||
add(b, 0, b.length);
|
||||
}
|
||||
|
||||
public void add(byte[] b, int off, int len) throws SQLException {
|
||||
if (len == 0) {
|
||||
return;
|
||||
}
|
||||
int newcount = size + len;
|
||||
checkSize(newcount);
|
||||
System.arraycopy(b, off, bytes, size, len);
|
||||
size = newcount;
|
||||
}
|
||||
|
||||
public void add(String string) throws SQLException {
|
||||
if (string == null) {
|
||||
return;
|
||||
}
|
||||
add(string.getBytes(StandardCharsets.UTF_8));
|
||||
}
|
||||
|
||||
private void checkSize(int newcount) throws SQLException {
|
||||
if (newcount > bytes.length) {
|
||||
bytes = ArrayUtils.grow(bytes, newcount);
|
||||
}
|
||||
}
|
||||
|
||||
public void writeTo(OutputStream out) throws IOException {
|
||||
out.write(bytes, offset, size);
|
||||
out.flush();
|
||||
}
|
||||
}
|
|
@ -1,239 +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;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.sql.jdbc.util;
|
||||
|
||||
import java.io.InputStream;
|
||||
|
||||
public class FastByteArrayInputStream extends InputStream {
|
||||
|
||||
/**
|
||||
* An array of bytes that was provided
|
||||
* by the creator of the stream. Elements <code>buf[0]</code>
|
||||
* through <code>buf[count-1]</code> are the
|
||||
* only bytes that can ever be read from the
|
||||
* stream; element <code>buf[pos]</code> is
|
||||
* the next byte to be read.
|
||||
*/
|
||||
protected BytesArray data;
|
||||
|
||||
/**
|
||||
* The index of the next character to read from the input stream buffer.
|
||||
* This value should always be nonnegative
|
||||
* and not larger than the value of <code>count</code>.
|
||||
* The next byte to be read from the input stream buffer
|
||||
* will be <code>buf[pos]</code>.
|
||||
*/
|
||||
protected int pos;
|
||||
|
||||
/**
|
||||
* The currently marked position in the stream.
|
||||
* ByteArrayInputStream objects are marked at position zero by
|
||||
* default when constructed. They may be marked at another
|
||||
* position within the buffer by the <code>mark()</code> method.
|
||||
* The current buffer position is set to this point by the
|
||||
* <code>reset()</code> method.
|
||||
* <p>
|
||||
* If no mark has been set, then the value of mark is the offset
|
||||
* passed to the constructor (or 0 if the offset was not supplied).
|
||||
*
|
||||
* @since JDK1.1
|
||||
*/
|
||||
protected int mark = 0;
|
||||
|
||||
/**
|
||||
* The index one greater than the last valid character in the input
|
||||
* stream buffer.
|
||||
* This value should always be nonnegative
|
||||
* and not larger than the length of <code>buf</code>.
|
||||
* It is one greater than the position of
|
||||
* the last byte within <code>buf</code> that
|
||||
* can ever be read from the input stream buffer.
|
||||
*/
|
||||
protected int count;
|
||||
|
||||
public FastByteArrayInputStream(byte[] data) {
|
||||
this(new BytesArray(data));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a <code>ByteArrayInputStream</code>
|
||||
* so that it uses <code>buf</code> as its
|
||||
* buffer array.
|
||||
* The buffer array is not copied.
|
||||
* The initial value of <code>pos</code>
|
||||
* is <code>0</code> and the initial value
|
||||
* of <code>count</code> is the length of
|
||||
* <code>buf</code>.
|
||||
*
|
||||
* @param data the input buffer.
|
||||
*/
|
||||
public FastByteArrayInputStream(BytesArray data) {
|
||||
this.data = data;
|
||||
this.pos = 0;
|
||||
this.count = data.size;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the next byte of data from this input stream. The value
|
||||
* byte is returned as an <code>int</code> in the range
|
||||
* <code>0</code> to <code>255</code>. If no byte is available
|
||||
* because the end of the stream has been reached, the value
|
||||
* <code>-1</code> is returned.
|
||||
* <p>
|
||||
* This <code>read</code> method
|
||||
* cannot block.
|
||||
*
|
||||
* @return the next byte of data, or <code>-1</code> if the end of the
|
||||
* stream has been reached.
|
||||
*/
|
||||
public int read() {
|
||||
return (pos < count) ? (data.bytes[pos++] & 0xff) : -1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads up to <code>len</code> bytes of data into an array of bytes
|
||||
* from this input stream.
|
||||
* If <code>pos</code> equals <code>count</code>,
|
||||
* then <code>-1</code> is returned to indicate
|
||||
* end of file. Otherwise, the number <code>k</code>
|
||||
* of bytes read is equal to the smaller of
|
||||
* <code>len</code> and <code>count-pos</code>.
|
||||
* If <code>k</code> is positive, then bytes
|
||||
* <code>buf[pos]</code> through <code>buf[pos+k-1]</code>
|
||||
* are copied into <code>b[off]</code> through
|
||||
* <code>b[off+k-1]</code> in the manner performed
|
||||
* by <code>System.arraycopy</code>. The
|
||||
* value <code>k</code> is added into <code>pos</code>
|
||||
* and <code>k</code> is returned.
|
||||
* <p>
|
||||
* This <code>read</code> method cannot block.
|
||||
*
|
||||
* @param b the buffer into which the data is read.
|
||||
* @param off the start offset in the destination array <code>b</code>
|
||||
* @param len the maximum number of bytes read.
|
||||
* @return the total number of bytes read into the buffer, or
|
||||
* <code>-1</code> if there is no more data because the end of
|
||||
* the stream has been reached.
|
||||
* @throws NullPointerException If <code>b</code> is <code>null</code>.
|
||||
* @throws IndexOutOfBoundsException If <code>off</code> is negative,
|
||||
* <code>len</code> is negative, or <code>len</code> is greater than
|
||||
* <code>b.length - off</code>
|
||||
*/
|
||||
public int read(byte b[], int off, int len) {
|
||||
if (b == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
else if (off < 0 || len < 0 || len > b.length - off) {
|
||||
throw new IndexOutOfBoundsException();
|
||||
}
|
||||
if (pos >= count) {
|
||||
return -1;
|
||||
}
|
||||
if (pos + len > count) {
|
||||
len = count - pos;
|
||||
}
|
||||
if (len <= 0) {
|
||||
return 0;
|
||||
}
|
||||
System.arraycopy(data.bytes, pos, b, off, len);
|
||||
pos += len;
|
||||
return len;
|
||||
}
|
||||
|
||||
/**
|
||||
* Skips <code>n</code> bytes of input from this input stream. Fewer
|
||||
* bytes might be skipped if the end of the input stream is reached.
|
||||
* The actual number <code>k</code>
|
||||
* of bytes to be skipped is equal to the smaller
|
||||
* of <code>n</code> and <code>count-pos</code>.
|
||||
* The value <code>k</code> is added into <code>pos</code>
|
||||
* and <code>k</code> is returned.
|
||||
*
|
||||
* @param n the number of bytes to be skipped.
|
||||
* @return the actual number of bytes skipped.
|
||||
*/
|
||||
public long skip(long n) {
|
||||
if (pos + n > count) {
|
||||
n = count - pos;
|
||||
}
|
||||
if (n < 0) {
|
||||
return 0;
|
||||
}
|
||||
pos += n;
|
||||
return n;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of remaining bytes that can be read (or skipped over)
|
||||
* from this input stream.
|
||||
* <p>
|
||||
* The value returned is <code>count - pos</code>,
|
||||
* which is the number of bytes remaining to be read from the input buffer.
|
||||
*
|
||||
* @return the number of remaining bytes that can be read (or skipped
|
||||
* over) from this input stream without blocking.
|
||||
*/
|
||||
public int available() {
|
||||
return count - pos;
|
||||
}
|
||||
|
||||
public int position() {
|
||||
return pos;
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests if this <code>InputStream</code> supports mark/reset. The
|
||||
* <code>markSupported</code> method of <code>ByteArrayInputStream</code>
|
||||
* always returns <code>true</code>.
|
||||
*
|
||||
* @since JDK1.1
|
||||
*/
|
||||
public boolean markSupported() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the current marked position in the stream.
|
||||
* ByteArrayInputStream objects are marked at position zero by
|
||||
* default when constructed. They may be marked at another
|
||||
* position within the buffer by this method.
|
||||
* <p>
|
||||
* If no mark has been set, then the value of the mark is the
|
||||
* offset passed to the constructor (or 0 if the offset was not
|
||||
* supplied).
|
||||
* <p> Note: The <code>readAheadLimit</code> for this class
|
||||
* has no meaning.
|
||||
*
|
||||
* @since JDK1.1
|
||||
*/
|
||||
public void mark(int readAheadLimit) {
|
||||
mark = pos;
|
||||
}
|
||||
|
||||
/**
|
||||
* Resets the buffer to the marked position. The marked position
|
||||
* is 0 unless another position was marked or an offset was specified
|
||||
* in the constructor.
|
||||
*/
|
||||
public void reset() {
|
||||
pos = mark;
|
||||
}
|
||||
|
||||
/**
|
||||
* Closing a <tt>ByteArrayInputStream</tt> has no effect. The methods in
|
||||
* this class can be called after the stream has been closed without
|
||||
* generating an <tt>IOException</tt>.
|
||||
*/
|
||||
public void close() {}
|
||||
|
||||
public BytesArray bytes() {
|
||||
return data;
|
||||
}
|
||||
|
||||
public void setBytes(byte[] data, int size) {
|
||||
this.data.bytes(data, size);
|
||||
}
|
||||
}
|
|
@ -78,11 +78,12 @@ public abstract class Version {
|
|||
String ver = "Unknown";
|
||||
String hash = ver;
|
||||
|
||||
if (urlStr.startsWith("file:/") && (urlStr.endsWith(".jar") || urlStr.endsWith("-SNAPSHOT.jar"))) {
|
||||
if (urlStr.startsWith("file:/") && urlStr.endsWith("-SNAPSHOT.jar")) {
|
||||
try (JarInputStream jar = new JarInputStream(url.openStream())) {
|
||||
Manifest manifest = jar.getManifest();
|
||||
hash = manifest.getMainAttributes().getValue("Change");
|
||||
int[] vers = from(manifest.getMainAttributes().getValue("X-Compile-Elasticsearch-Version"));
|
||||
ver = manifest.getMainAttributes().getValue("X-Compile-Elasticsearch-Version");
|
||||
int[] vers = from(ver);
|
||||
maj = vers[0];
|
||||
min = vers[1];
|
||||
rev = vers[2];
|
||||
|
@ -128,4 +129,4 @@ public abstract class Version {
|
|||
public static int jdbcMinorVersion() {
|
||||
return 2;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -8,12 +8,13 @@ package org.elasticsearch.xpack.sql.jdbc.util;
|
|||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
public class VersionTests extends ESTestCase {
|
||||
public void testVersionIsUnknownWithoutAJar() {
|
||||
// We aren't running in a jar so we have a bunch of "Unknown"
|
||||
assertEquals("Unknown", Version.versionNumber());
|
||||
assertEquals("Unknown", Version.versionHash());
|
||||
assertEquals(0, Version.versionMajor());
|
||||
assertEquals(0, Version.versionMinor());
|
||||
public void testVersionIsCurrent() {
|
||||
/* This test will only work properly in gradle because in gradle we run the tests
|
||||
* using the jar. */
|
||||
assertEquals(org.elasticsearch.Version.CURRENT.toString(), Version.versionNumber());
|
||||
assertNotNull(Version.versionHash());
|
||||
assertEquals(org.elasticsearch.Version.CURRENT.major, Version.versionMajor());
|
||||
assertEquals(org.elasticsearch.Version.CURRENT.minor, Version.versionMinor());
|
||||
}
|
||||
|
||||
public void test70Version() {
|
||||
|
|
|
@ -20,6 +20,7 @@ import org.elasticsearch.rest.RestRequest;
|
|||
import org.elasticsearch.rest.RestRequest.Method;
|
||||
import org.elasticsearch.xpack.sql.ClientSqlException;
|
||||
import org.elasticsearch.xpack.sql.analysis.AnalysisException;
|
||||
import org.elasticsearch.xpack.sql.analysis.catalog.MappingException;
|
||||
import org.elasticsearch.xpack.sql.jdbc.net.protocol.Proto;
|
||||
import org.elasticsearch.xpack.sql.parser.ParsingException;
|
||||
import org.elasticsearch.xpack.sql.planner.PlanningException;
|
||||
|
@ -124,6 +125,9 @@ public abstract class AbstractSqlProtocolRestAction extends BaseRestHandler {
|
|||
if (cause instanceof AnalysisException || cause instanceof ResourceNotFoundException) {
|
||||
return SqlExceptionType.DATA;
|
||||
}
|
||||
if (cause instanceof PlanningException || cause instanceof MappingException) {
|
||||
return SqlExceptionType.NOT_SUPPORTED;
|
||||
}
|
||||
if (cause instanceof ParsingException) {
|
||||
return SqlExceptionType.SYNTAX;
|
||||
}
|
||||
|
|
|
@ -1,5 +1,13 @@
|
|||
description = 'Code shared between jdbc and cli'
|
||||
|
||||
dependencies {
|
||||
compile "com.fasterxml.jackson.core:jackson-core:${versions.jackson}"
|
||||
}
|
||||
|
||||
dependencyLicenses {
|
||||
mapping from: /jackson-.*/, to: 'jackson'
|
||||
}
|
||||
|
||||
forbiddenApisMain {
|
||||
// does not depend on core, so only jdk and http signatures should be checked
|
||||
signaturesURLs = [this.class.getResource('/forbidden/jdk-signatures.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
|
|
@ -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.
|
|
@ -0,0 +1 @@
|
|||
eb21a035c66ad307e66ec8fce37f5d50fd62d039
|
|
@ -1,86 +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;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.sql.client.shared;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.util.Arrays;
|
||||
|
||||
public class BasicByteArrayOutputStream extends OutputStream {
|
||||
|
||||
protected byte buf[];
|
||||
|
||||
protected int count;
|
||||
|
||||
BasicByteArrayOutputStream() {
|
||||
this(32);
|
||||
}
|
||||
|
||||
BasicByteArrayOutputStream(int size) {
|
||||
if (size < 0) {
|
||||
throw new IllegalArgumentException("Negative initial size: " + size);
|
||||
}
|
||||
buf = new byte[size];
|
||||
}
|
||||
|
||||
private void ensureCapacity(int minCapacity) {
|
||||
// overflow-conscious code
|
||||
if (minCapacity - buf.length > 0)
|
||||
grow(minCapacity);
|
||||
}
|
||||
|
||||
private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
|
||||
|
||||
private void grow(int minCapacity) {
|
||||
// overflow-conscious code
|
||||
int oldCapacity = buf.length;
|
||||
int newCapacity = oldCapacity << 1;
|
||||
if (newCapacity - minCapacity < 0)
|
||||
newCapacity = minCapacity;
|
||||
if (newCapacity - MAX_ARRAY_SIZE > 0)
|
||||
newCapacity = hugeCapacity(minCapacity);
|
||||
buf = Arrays.copyOf(buf, newCapacity);
|
||||
}
|
||||
|
||||
private static int hugeCapacity(int minCapacity) {
|
||||
if (minCapacity < 0) // overflow
|
||||
throw new OutOfMemoryError();
|
||||
return (minCapacity > MAX_ARRAY_SIZE) ? Integer.MAX_VALUE : MAX_ARRAY_SIZE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(int b) {
|
||||
ensureCapacity(count + 1);
|
||||
buf[count] = (byte) b;
|
||||
count += 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(byte b[], int off, int len) {
|
||||
if ((off < 0) || (off > b.length) || (len < 0) || ((off + len) - b.length > 0)) {
|
||||
throw new IndexOutOfBoundsException();
|
||||
}
|
||||
ensureCapacity(count + len);
|
||||
System.arraycopy(b, off, buf, count, len);
|
||||
count += len;
|
||||
}
|
||||
|
||||
public void writeTo(OutputStream out) throws IOException {
|
||||
out.write(buf, 0, count);
|
||||
}
|
||||
|
||||
public void reset() {
|
||||
count = 0;
|
||||
}
|
||||
|
||||
public int size() {
|
||||
return count;
|
||||
}
|
||||
|
||||
public Bytes bytesArray() {
|
||||
return new Bytes(buf, count);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.sql.client.shared;
|
||||
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* A {@link Function}-like interface which allows throwing checked exceptions.
|
||||
* Elasticsearch has one of these but we don't depend on Elasticsearch.
|
||||
*/
|
||||
@FunctionalInterface
|
||||
public interface CheckedFunction<T, R, E extends Exception> {
|
||||
R apply(T t) throws E;
|
||||
}
|
|
@ -1,26 +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;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.sql.client.shared;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
||||
public class IOUtils {
|
||||
public static Bytes asBytes(InputStream input) throws IOException {
|
||||
BasicByteArrayOutputStream bos = new BasicByteArrayOutputStream(input.available());
|
||||
byte[] buffer = new byte[1024];
|
||||
int read = 0;
|
||||
try (InputStream in = input) {
|
||||
while ((read = in.read(buffer)) != -1) {
|
||||
bos.write(buffer, 0, read);
|
||||
}
|
||||
} finally {
|
||||
// non needed but used to avoid the warnings
|
||||
bos.close();
|
||||
}
|
||||
return bos.bytesArray();
|
||||
}
|
||||
}
|
|
@ -6,6 +6,8 @@
|
|||
package org.elasticsearch.xpack.sql.client.shared;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutput;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
|
@ -17,6 +19,7 @@ import java.net.URL;
|
|||
import java.security.AccessController;
|
||||
import java.security.PrivilegedAction;
|
||||
import java.util.Base64;
|
||||
import java.util.function.BiFunction;
|
||||
import java.util.function.Function;
|
||||
import java.util.zip.GZIPInputStream;
|
||||
|
||||
|
@ -99,7 +102,11 @@ public class JreHttpUrlConnection implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
public Bytes post(CheckedConsumer<DataOutput, IOException> doc) throws ClientException {
|
||||
public <R> R post(
|
||||
CheckedConsumer<DataOutput, IOException> doc,
|
||||
CheckedFunction<DataInput, R, IOException> parser,
|
||||
BiFunction<Integer, RemoteFailure, R> failureConverter
|
||||
) throws ClientException {
|
||||
try {
|
||||
con.setRequestMethod("POST");
|
||||
con.setDoOutput(true);
|
||||
|
@ -107,15 +114,16 @@ public class JreHttpUrlConnection implements Closeable {
|
|||
try (OutputStream out = con.getOutputStream()) {
|
||||
doc.accept(new DataOutputStream(out));
|
||||
}
|
||||
if (con.getResponseCode() >= 500) {
|
||||
throw new ClientException("Server error: %s(%d;%s)", con.getResponseMessage(), con.getResponseCode(),
|
||||
IOUtils.asBytes(getStream(con, con.getErrorStream())).toString());
|
||||
if (con.getResponseCode() < 300) {
|
||||
try (InputStream stream = getStream(con, con.getInputStream())) {
|
||||
return parser.apply(new DataInputStream(stream));
|
||||
}
|
||||
}
|
||||
if (con.getResponseCode() >= 400) {
|
||||
throw new ClientException("Client error: %s(%d;%s)", con.getResponseMessage(), con.getResponseCode(),
|
||||
IOUtils.asBytes(getStream(con, con.getErrorStream())).toString());
|
||||
RemoteFailure failure;
|
||||
try (InputStream stream = getStream(con, con.getErrorStream())) {
|
||||
failure = RemoteFailure.parseFromResponse(stream);
|
||||
}
|
||||
return IOUtils.asBytes(getStream(con, con.getInputStream()));
|
||||
return failureConverter.apply(con.getResponseCode(), failure);
|
||||
} catch (IOException ex) {
|
||||
throw new ClientException(ex, "Cannot POST address %s (%s)", url, ex.getMessage());
|
||||
}
|
||||
|
@ -173,4 +181,4 @@ public class JreHttpUrlConnection implements Closeable {
|
|||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,219 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.sql.client.shared;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonFactory;
|
||||
import com.fasterxml.jackson.core.JsonGenerator;
|
||||
import com.fasterxml.jackson.core.JsonParser;
|
||||
import com.fasterxml.jackson.core.JsonToken;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
||||
|
||||
/**
|
||||
* A failure that happened on the remote server.
|
||||
*/
|
||||
public class RemoteFailure {
|
||||
private static final JsonFactory JSON_FACTORY = new JsonFactory();
|
||||
static {
|
||||
// Set up the factory similarly to how XContent does
|
||||
JSON_FACTORY.configure(JsonGenerator.Feature.QUOTE_FIELD_NAMES, true);
|
||||
JSON_FACTORY.configure(JsonParser.Feature.ALLOW_COMMENTS, true);
|
||||
JSON_FACTORY.configure(JsonFactory.Feature.FAIL_ON_SYMBOL_HASH_OVERFLOW, true);
|
||||
// Do not automatically close unclosed objects/arrays in com.fasterxml.jackson.core.json.UTF8JsonGenerator#close() method
|
||||
JSON_FACTORY.configure(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT, false);
|
||||
JSON_FACTORY.configure(JsonParser.Feature.STRICT_DUPLICATE_DETECTION, false);
|
||||
|
||||
}
|
||||
public static RemoteFailure parseFromResponse(InputStream stream) throws IOException {
|
||||
try (JsonParser parser = JSON_FACTORY.createParser(stream)) {
|
||||
try {
|
||||
return parseResponseTopLevel(parser);
|
||||
} catch (IOException e) {
|
||||
throw new IOException(
|
||||
"Can't parse error from Elasticearch [" + e.getMessage() + "] at [line "
|
||||
+ parser.getTokenLocation().getLineNr() + " col " + parser.getTokenLocation().getColumnNr() + "]",
|
||||
e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private final String type;
|
||||
private final String reason;
|
||||
private final String remoteTrace;
|
||||
private final Map<String, String> headers;
|
||||
private final RemoteFailure cause;
|
||||
|
||||
RemoteFailure(String type, String reason, String remoteTrace, Map<String, String> headers, RemoteFailure cause) {
|
||||
this.type = type;
|
||||
this.reason = reason;
|
||||
this.remoteTrace = remoteTrace;
|
||||
this.headers = headers;
|
||||
this.cause = cause;
|
||||
}
|
||||
|
||||
public String type() {
|
||||
return type;
|
||||
}
|
||||
|
||||
public String reason() {
|
||||
return reason;
|
||||
}
|
||||
|
||||
/**
|
||||
* Stack trace from Elasticsearch for the remote failure. Mostly just useful for debugging
|
||||
* errors that happen to be bugs.
|
||||
*/
|
||||
public String remoteTrace() {
|
||||
return remoteTrace;
|
||||
}
|
||||
|
||||
/**
|
||||
* Headers sent by the remote failure.
|
||||
*/
|
||||
public Map<String, String> headers() {
|
||||
return headers;
|
||||
}
|
||||
|
||||
/**
|
||||
* Cause of the remote failure. Mostly just useful for dbuegging errors that happen to be bugs.
|
||||
*/
|
||||
public RemoteFailure cause() {
|
||||
return cause;
|
||||
}
|
||||
|
||||
private static RemoteFailure parseResponseTopLevel(JsonParser parser) throws IOException {
|
||||
RemoteFailure exception = null;
|
||||
|
||||
/* It'd be lovely to use the high level constructs that we have in core like ObjectParser
|
||||
* but, alas, we aren't going to modularize those out any time soon. */
|
||||
JsonToken token = parser.nextToken();
|
||||
if (token != JsonToken.START_OBJECT) {
|
||||
throw new IllegalArgumentException("Expected error to start with [START_OBJECT] but started with [" + token
|
||||
+ "][" + parser.getText() + "]");
|
||||
}
|
||||
String fieldName = null;
|
||||
while ((token = parser.nextToken()) != JsonToken.END_OBJECT) {
|
||||
if (token == JsonToken.FIELD_NAME) {
|
||||
fieldName = parser.getCurrentName();
|
||||
} else {
|
||||
switch (fieldName) {
|
||||
case "error":
|
||||
if (token != JsonToken.START_OBJECT) {
|
||||
throw new IOException("Expected [error] to be an object but was [" + token + "][" + parser.getText() + "]");
|
||||
}
|
||||
exception = parseFailure(parser);
|
||||
continue;
|
||||
case "status":
|
||||
if (token != JsonToken.VALUE_NUMBER_INT) {
|
||||
throw new IOException("Expected [status] to be a string but was [" + token + "][" + parser.getText() + "]");
|
||||
}
|
||||
// Intentionally ignored
|
||||
continue;
|
||||
default:
|
||||
throw new IOException("Expected one of [error, status] but got [" + fieldName + "][" + parser.getText() + "]");
|
||||
}
|
||||
}
|
||||
}
|
||||
if (exception == null) {
|
||||
throw new IOException("Expected [error] but didn't see it.");
|
||||
}
|
||||
return exception;
|
||||
}
|
||||
|
||||
private static RemoteFailure parseFailure(JsonParser parser) throws IOException {
|
||||
String type = null;
|
||||
String reason = null;
|
||||
String remoteTrace = null;
|
||||
Map<String, String> headers = emptyMap();
|
||||
RemoteFailure cause = null;
|
||||
|
||||
JsonToken token;
|
||||
String fieldName = null;
|
||||
while ((token = parser.nextToken()) != JsonToken.END_OBJECT) {
|
||||
if (token == JsonToken.FIELD_NAME) {
|
||||
fieldName = parser.getCurrentName();
|
||||
} else {
|
||||
switch (fieldName) {
|
||||
case "caused_by":
|
||||
if (token != JsonToken.START_OBJECT) {
|
||||
throw new IOException("Expected [caused_by] to be an object but was [" + token + "][" + parser.getText() + "]");
|
||||
}
|
||||
cause = parseFailure(parser);
|
||||
break;
|
||||
case "header":
|
||||
if (token != JsonToken.START_OBJECT) {
|
||||
throw new IOException("Expected [header] to be an object but was [" + token + "][" + parser.getText() + "]");
|
||||
}
|
||||
headers = parseHeaders(parser);
|
||||
break;
|
||||
case "reason":
|
||||
switch (token) {
|
||||
case VALUE_STRING:
|
||||
reason = parser.getText();
|
||||
break;
|
||||
case VALUE_NULL:
|
||||
break;
|
||||
default:
|
||||
throw new IOException("Expected [reason] to be a string but was [" + token + "][" + parser.getText() + "]");
|
||||
}
|
||||
break;
|
||||
case "root_cause":
|
||||
if (token != JsonToken.START_ARRAY) {
|
||||
throw new IOException("Expected [root_cause] to be an array but was [" + token + "][" + parser.getText() + "]");
|
||||
}
|
||||
parser.skipChildren(); // Intentionally ignored
|
||||
break;
|
||||
case "stack_trace":
|
||||
if (token != JsonToken.VALUE_STRING) {
|
||||
throw new IOException("Expected [stack_trace] to be a string but was [" + token + "][" + parser.getText() + "]");
|
||||
}
|
||||
remoteTrace = parser.getText();
|
||||
break;
|
||||
case "type":
|
||||
if (token != JsonToken.VALUE_STRING) {
|
||||
throw new IOException("Expected [type] to be a string but was [" + token + "][" + parser.getText() + "]");
|
||||
}
|
||||
type = parser.getText();
|
||||
break;
|
||||
default:
|
||||
throw new IOException("Expected one of [caused_by, reason, root_cause, stack_trace, type] but got ["
|
||||
+ fieldName + "]");
|
||||
}
|
||||
}
|
||||
}
|
||||
if (type == null) {
|
||||
throw new IOException("expected [type] but didn't see it");
|
||||
}
|
||||
if (remoteTrace == null) {
|
||||
throw new IOException("expected [stack_trace] cannot but didn't see it");
|
||||
}
|
||||
return new RemoteFailure(type, reason, remoteTrace, headers, cause);
|
||||
}
|
||||
|
||||
private static Map<String, String> parseHeaders(JsonParser parser) throws IOException {
|
||||
Map<String, String> headers = new HashMap<>();
|
||||
|
||||
JsonToken token;
|
||||
while ((token = parser.nextToken()) != JsonToken.END_OBJECT) {
|
||||
if (token != JsonToken.FIELD_NAME) {
|
||||
throw new IOException("expected header name but was [" + token + "][" + parser.getText() + "]");
|
||||
}
|
||||
String name = parser.getText();
|
||||
token = parser.nextToken();
|
||||
if (token != JsonToken.VALUE_STRING) {
|
||||
throw new IOException("expected header value but was [" + token + "][" + parser.getText() + "]");
|
||||
}
|
||||
String value = parser.getText();
|
||||
headers.put(name, value);
|
||||
}
|
||||
|
||||
return headers;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,90 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.sql.client.shared;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.file.Files;
|
||||
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.singletonMap;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
|
||||
public class RemoteFailureTests extends ESTestCase {
|
||||
public void testParseBasic() throws IOException {
|
||||
RemoteFailure failure = parse("basic.json");
|
||||
assertEquals("illegal_argument_exception", failure.type());
|
||||
assertEquals("[sql/query] unknown field [test], parser not found", failure.reason());
|
||||
assertThat(failure.remoteTrace(),
|
||||
containsString("at org.elasticsearch.common.xcontent.ObjectParser.getParser(ObjectParser.java:346)"));
|
||||
assertNull(failure.cause());
|
||||
assertEquals(emptyMap(), failure.headers());
|
||||
}
|
||||
|
||||
public void testParseNested() throws IOException {
|
||||
RemoteFailure failure = parse("nested.json");
|
||||
assertEquals("parsing_exception", failure.type());
|
||||
assertEquals("line 1:1: no viable alternative at input 'test'", failure.reason());
|
||||
assertThat(failure.remoteTrace(),
|
||||
containsString("at org.elasticsearch.xpack.sql.parser.SqlParser$1.syntaxError(SqlParser.java:151)"));
|
||||
assertNotNull(failure.cause());
|
||||
|
||||
failure = failure.cause();
|
||||
assertEquals("no_viable_alt_exception", failure.type());
|
||||
assertEquals(null, failure.reason());
|
||||
assertThat(failure.remoteTrace(),
|
||||
containsString("at org.antlr.v4.runtime.atn.ParserATNSimulator.noViableAlt(ParserATNSimulator.java:1886)"));
|
||||
assertNull(failure.cause());
|
||||
assertEquals(emptyMap(), failure.headers());
|
||||
}
|
||||
|
||||
public void testParseMissingAuth() throws IOException {
|
||||
RemoteFailure failure = parse("missing_auth.json");
|
||||
assertEquals("security_exception", failure.type());
|
||||
assertEquals("missing authentication token for REST request [/?pretty&error_trace]", failure.reason());
|
||||
assertThat(failure.remoteTrace(),
|
||||
containsString("DefaultAuthenticationFailureHandler.missingToken"));
|
||||
assertNull(failure.cause());
|
||||
assertEquals(singletonMap("WWW-Authenticate", "Basic realm=\"security\" charset=\"UTF-8\""),
|
||||
failure.headers());
|
||||
}
|
||||
|
||||
public void testNoError() throws IOException {
|
||||
IOException e = expectThrows(IOException.class, () -> parse("no_error.json"));
|
||||
assertEquals(
|
||||
"Can't parse error from Elasticearch [Expected [error] but didn't see it.] at [line 2 col 1]",
|
||||
e.getMessage());
|
||||
}
|
||||
|
||||
public void testBogusError() throws IOException {
|
||||
IOException e = expectThrows(IOException.class, () -> parse("bogus_error.json"));
|
||||
assertEquals(
|
||||
"Can't parse error from Elasticearch [Expected [error] to be an object but was [VALUE_STRING][bogus]] at [line 2 col 12]",
|
||||
e.getMessage());
|
||||
}
|
||||
|
||||
public void testNoStack() throws IOException {
|
||||
IOException e = expectThrows(IOException.class, () -> parse("no_stack.json"));
|
||||
assertEquals(
|
||||
"Can't parse error from Elasticearch [expected [stack_trace] cannot but didn't see it] at [line 5 col 3]",
|
||||
e.getMessage());
|
||||
}
|
||||
|
||||
public void testNoType() throws IOException {
|
||||
IOException e = expectThrows(IOException.class, () -> parse("no_type.json"));
|
||||
assertEquals(
|
||||
"Can't parse error from Elasticearch [expected [type] but didn't see it] at [line 5 col 3]",
|
||||
e.getMessage());
|
||||
}
|
||||
|
||||
private RemoteFailure parse(String fileName) throws IOException {
|
||||
try (InputStream in = Files.newInputStream(getDataPath("/remote_failure/" + fileName))) {
|
||||
return RemoteFailure.parseFromResponse(in);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -13,8 +13,7 @@ import org.junit.Before;
|
|||
import org.junit.ClassRule;
|
||||
import org.junit.rules.ExternalResource;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataInput;
|
||||
import java.net.URL;
|
||||
import java.security.AccessController;
|
||||
import java.security.PrivilegedAction;
|
||||
|
@ -107,13 +106,16 @@ public class SSLTests extends ESTestCase {
|
|||
|
||||
public void testSslPost() throws Exception {
|
||||
String message = UUID.randomUUID().toString();
|
||||
Bytes b = AccessController.doPrivileged((PrivilegedAction<Bytes>) () ->
|
||||
String received = AccessController.doPrivileged((PrivilegedAction<String>) () ->
|
||||
JreHttpUrlConnection.http(sslServer, cfg, c ->
|
||||
c.post(o -> {
|
||||
o.writeUTF(message);
|
||||
})));
|
||||
c.post(
|
||||
out -> out.writeUTF(message),
|
||||
DataInput::readUTF,
|
||||
(status, failure) -> "failure: [" + status + "][" + failure + "]"
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
String received = new DataInputStream(new ByteArrayInputStream(b.bytes())).readUTF();
|
||||
assertEquals(message, received);
|
||||
}
|
||||
}
|
File diff suppressed because one or more lines are too long
|
@ -0,0 +1,3 @@
|
|||
{
|
||||
"error": "bogus"
|
||||
}
|
File diff suppressed because one or more lines are too long
File diff suppressed because one or more lines are too long
|
@ -0,0 +1,2 @@
|
|||
{
|
||||
}
|
|
@ -0,0 +1,6 @@
|
|||
{
|
||||
"error" : {
|
||||
"type" : "illegal_argument_exception",
|
||||
"reason" : "[sql/query] unknown field [test], parser not found"
|
||||
}
|
||||
}
|
File diff suppressed because one or more lines are too long
|
@ -24,7 +24,7 @@ import static java.util.Collections.emptyMap;
|
|||
/**
|
||||
* Base implementation for the binary protocol for the CLI and JDBC.
|
||||
* All backwards compatibility is done on the server side using the
|
||||
* version number sent in the header.
|
||||
* version number sent in the header.
|
||||
*/
|
||||
public abstract class AbstractProto {
|
||||
private static final int MAGIC_NUMBER = 0x0C0DEC110;
|
||||
|
@ -87,6 +87,23 @@ public abstract class AbstractProto {
|
|||
TIMEOUT(SQLTimeoutException::new),
|
||||
NOT_SUPPORTED(SQLFeatureNotSupportedException::new);
|
||||
|
||||
public static SqlExceptionType fromRemoteFailureType(String type) {
|
||||
switch (type) {
|
||||
case "analysis":
|
||||
case "resouce_not_found":
|
||||
return DATA;
|
||||
case "planning":
|
||||
case "mapping":
|
||||
return NOT_SUPPORTED;
|
||||
case "parsing":
|
||||
return SYNTAX;
|
||||
case "timeout":
|
||||
return TIMEOUT;
|
||||
default:
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private final Function<String, SQLException> toException;
|
||||
|
||||
SqlExceptionType(Function<String, SQLException> toException) {
|
||||
|
@ -140,7 +157,7 @@ public abstract class AbstractProto {
|
|||
|
||||
/**
|
||||
* Read the protocol header.
|
||||
* @return the version
|
||||
* @return the version
|
||||
* @throws IOException if there is an underlying {@linkplain IOException} or if the protocol is malformed
|
||||
*/
|
||||
private static int readHeader(DataInput in) throws IOException {
|
||||
|
@ -151,4 +168,4 @@ public abstract class AbstractProto {
|
|||
int version = in.readInt();
|
||||
return version;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.
|
|
@ -1,5 +0,0 @@
|
|||
Elasticsearch
|
||||
Copyright 2009-2017 Elasticsearch
|
||||
|
||||
This product includes software developed by The Apache Software
|
||||
Foundation (http://www.apache.org/).
|
Loading…
Reference in New Issue