All configuration properties start with opensearch
prefix. Note that the opensearch.internal
namespace is reserved for the library internal use and should not be used by the user at any point.
The properties are read mainly from the Hadoop configuration but the user can specify (some of) them directly depending on the library used.
opensearch.resource=<OpenSearch resource location, relative to the host/port specified above>
opensearch.query=<uri or query dsl query> # defaults to {"query":{"match_all":{}}}
opensearch.nodes=<OpenSearch host address> # defaults to localhost
opensearch.port=<OPENSEARCH REST port> # defaults to 9200
For basic, low-level or performance-sensitive environments, OpenSearch-Hadoop provides dedicated InputFormat
and OutputFormat
that read and write data to OpenSearch. To use them, add the opensearch-hadoop
jar to your job classpath
(either by bundling the library along - it's ~300kB and there are no-dependencies), using the DistributedCache or by provisioning the cluster manually.
Note that os-hadoop supports both the so-called 'old' and the 'new' API through its OpenSearchInputFormat
and OpenSearchOutputFormat
classes.
To read data from OpenSearch, configure the OpenSearchInputFormat
on your job configuration along with the relevant properties:
JobConf conf = new JobConf();
conf.setInputFormat(OpenSearchInputFormat.class);
conf.set("opensearch.resource", "radio/artists");
conf.set("opensearch.query", "?q=me*"); // replace this with the relevant query
...
JobClient.runJob(conf);
Same configuration template can be used for writing but using OpenSearchOuputFormat
:
JobConf conf = new JobConf();
conf.setOutputFormat(OpenSearchOutputFormat.class);
conf.set("opensearch.resource", "radio/artists"); // index or indices used for storing data
...
JobClient.runJob(conf);
Configuration conf = new Configuration();
conf.set("opensearch.resource", "radio/artists");
conf.set("opensearch.query", "?q=me*"); // replace this with the relevant query
Job job = new Job(conf)
job.setInputFormatClass(OpenSearchInputFormat.class);
...
job.waitForCompletion(true);
Configuration conf = new Configuration();
conf.set("opensearch.resource", "radio/artists"); // index or indices used for storing data
Job job = new Job(conf)
job.setOutputFormatClass(OpenSearchOutputFormat.class);
...
job.waitForCompletion(true);
Signing requests would require the aws-sdk-bundle in your job classpath.
Configuration conf = new Configuration();
conf.set("opensearch.resource", "radio/artists");
conf.set("opensearch.query", "?q=me*"); // replace this with the relevant query
conf.set("opensearch.nodes", "https://search-xxx.us-east-1.es.amazonaws.com");
conf.set("opensearch.port", "443");
conf.set("opensearch.net.ssl", "true");
conf.set("opensearch.nodes.wan.only", "true");
conf.set("opensearch.aws.sigv4.enabled", "true");
conf.set("opensearch.aws.sigv4.region", "us-east-1");
Job job = new Job(conf)
job.setInputFormatClass(OpenSearchInputFormat.class);
...
job.waitForCompletion(true);
OpenSearch-Hadoop provides a Hive storage handler for OpenSearch, meaning one can define an external table on top of OpenSearch.
Add opensearch-hadoop-.jar to hive.aux.jars.path
or register it manually in your Hive script (recommended):
ADD JAR /path_to_jar/opensearch-hadoop-<version>.jar;
To read data from OpenSearch, define a table backed by the desired index:
CREATE EXTERNAL TABLE artists (
id BIGINT,
name STRING,
links STRUCT<url:STRING, picture:STRING>)
STORED BY 'org.opensearch.hadoop.hive.OpenSearchStorageHandler'
TBLPROPERTIES('opensearch.resource' = 'radio/artists', 'opensearch.query' = '?q=me*');
The fields defined in the table are mapped to the JSON when communicating with OpenSearch. Notice the use of TBLPROPERTIES
to define the location, that is the query used for reading from this table.
Once defined, the table can be used just like any other:
SELECT * FROM artists;
To write data, a similar definition is used but with a different opensearch.resource
:
CREATE EXTERNAL TABLE artists (
id BIGINT,
name STRING,
links STRUCT<url:STRING, picture:STRING>)
STORED BY 'org.opensearch.hadoop.hive.OpenSearchStorageHandler'
TBLPROPERTIES('opensearch.resource' = 'radio/artists');
Any data passed to the table is then passed down to OpenSearch; for example considering a table s
, mapped to a TSV/CSV file, one can index it to OpenSearch like this:
INSERT OVERWRITE TABLE artists
SELECT NULL, s.name, named_struct('url', s.url, 'picture', s.picture) FROM source s;
As one can note, currently the reading and writing are treated separately but we're working on unifying the two and automatically translating HiveQL to OpenSearch queries.
Signing requests would require the aws-sdk-bundle in your job classpath.
CREATE EXTERNAL TABLE artists (
id BIGINT,
name STRING,
links STRUCT<url:STRING, picture:STRING>)
STORED BY 'org.opensearch.hadoop.hive.OpenSearchStorageHandler'
TBLPROPERTIES(
'opensearch.nodes' = 'https://search-xxx.us-east-1.es.amazonaws.com',
'opensearch.port' = '443',
'opensearch.net.ssl' = 'true',
'opensearch.resource' = 'artists',
'opensearch.nodes.wan.only' = 'true',
'opensearch.aws.sigv4.enabled' = 'true',
'opensearch.aws.sigv4.region' = 'us-east-1'
);
OpenSearch-Hadoop provides native (Java and Scala) integration with Spark: for reading a dedicated RDD
and for writing, methods that work on any RDD
. Spark SQL is also supported
To read data from OpenSearch, create a dedicated RDD
and specify the query as an argument:
import org.opensearch.spark._
..
val conf = ...
val sc = new SparkContext(conf)
sc.opensearchRDD("radio/artists", "?q=me*")
import org.opensearch.spark.sql._
// DataFrame schema automatically inferred
val df = sqlContext.read.format("opensearch").load("buckethead/albums")
// operations get pushed down and translated at runtime to OpenSearch QueryDSL
val playlist = df.filter(df("category").equalTo("pikes").and(df("year").geq(2016)))
Import the org.opensearch.spark._
package to gain savetoOpenSearch
methods on your RDD
s:
import org.opensearch.spark._
val conf = ...
val sc = new SparkContext(conf)
val numbers = Map("one" -> 1, "two" -> 2, "three" -> 3)
val airports = Map("OTP" -> "Otopeni", "SFO" -> "San Fran")
sc.makeRDD(Seq(numbers, airports)).saveToOpenSearch("spark/docs")
import org.opensearch.spark.sql._
val df = sqlContext.read.json("examples/people.json")
df.saveToOpenSearch("spark/people")
In a Java environment, use the org.opensearch.spark.rdd.java.api
package, in particular the JavaOpenSearchSpark
class.
To read data from OpenSearch, create a dedicated RDD
and specify the query as an argument.
import org.apache.spark.api.java.JavaSparkContext;
import org.opensearch.spark.rdd.api.java.JavaOpenSearchSpark;
SparkConf conf = ...
JavaSparkContext jsc = new JavaSparkContext(conf);
JavaPairRDD<String, Map<String, Object>> opensearchRDD = JavaOpenSearchSpark.opensearchRDD(jsc, "radio/artists");
SQLContext sql = new SQLContext(sc);
DataFrame df = sql.read().format("opensearch").load("buckethead/albums");
DataFrame playlist = df.filter(df.col("category").equalTo("pikes").and(df.col("year").geq(2016)))
Use JavaOpenSearchSpark
to index any RDD
to OpenSearch:
import org.opensearch.spark.rdd.api.java.JavaOpenSearchSpark;
SparkConf conf = ...
JavaSparkContext jsc = new JavaSparkContext(conf);
Map<String, ?> numbers = ImmutableMap.of("one", 1, "two", 2);
Map<String, ?> airports = ImmutableMap.of("OTP", "Otopeni", "SFO", "San Fran");
JavaRDD<Map<String, ?>> javaRDD = jsc.parallelize(ImmutableList.of(numbers, airports));
JavaOpenSearchSpark.saveToOpenSearch(javaRDD, "spark/docs");
import org.opensearch.spark.sql.api.java.JavaOpenSearchSparkSQL;
DataFrame df = sqlContext.read.json("examples/people.json")
JavaOpenSearchSparkSQL.saveToOpenSearch(df, "spark/docs")
Signing requests would require the aws-sdk-bundle in your job classpath.
import org.apache.spark.sql.SQLContext
import org.apache.spark.sql.SQLContext._
import org.opensearch.spark.sql._
val sql = new SQLContext(sc)
val accountsRead = sql.read.json("/Users/user/release/maximus/data/accounts.json")
val options = Map("pushdown" -> "true",
"opensearch.nodes" -> "https://aos-2.3-domain",
"opensearch.aws.sigv4.enabled" -> "true",
"opensearch.aws.sigv4.region" -> "us-west-2",
"opensearch.nodes.resolve.hostname" -> "false",
"opensearch.nodes.wan.only" -> "true",
"opensearch.net.ssl" -> "true")
accountsRead.saveToOpenSearch("accounts-00001", options)