Mapping and Typesedit

As explained in the previous sections, elasticsearch-hadoop integrates closely with the Hadoop ecosystem and performs close introspection of the type information so that the data flow between Elasticsearch and Hadoop is as transparent as possible. This section takes a closer look at how the type conversion takes place and how data is mapped between the two systems.

Converting data to Elasticsearchedit

By design, elasticsearch-hadoop provides no data transformation or mapping layer itself simply because there is no need for them: Hadoop is designed to do ETL and some libraries (like Pig and Hive) provide type information themselves. Furthermore, Elasticsearch has rich support for mapping out of the box including automatic detection, dynamic/schema-less mapping, templates and full manual control. Need to split strings into token, do data validation or eliminate unneeded data? There are plenty of ways to do that in Hadoop before reading/writing data from/to Elasticsearch. Need control over how data is stored in Elasticsearch? Use Elasticsearch APIs to define the mapping, to update settings or add generic meta-data.

Time/Date mappingedit

When it comes to handling dates, Elasticsearch always uses the ISO 8601 format for date/time. This is the default date format of Elasticsearch - if a custom one is needed, please add it to the default option rather then just replacing it. See the date format section in Elasticsearch reference documentation for more information. Note that when reading data, if the date is not in ISO8601 format, by default elasticsearch-hadoop will likely not understand it as it does not replicate the elaborate date parsing in Elasticsearch. In these cases one can simply disable the date conversion and pass the raw information as a long or String, through the es.mapping.date.rich property.

As a side note, elasticsearch-hadoop tries to detect whether dedicated date parsing libraries (in particular Joda, used also by Elasticsearch) are available at runtime and if so, will use them. If not, it will default to parsing using JDK classes which are not as rich. Going forward especially with the advent of JDK 8, elasticsearch-hadoop will try to migrate to javax.time library to have the same behaviour regardless of the classpath available at runtime.

Ordering and Mappingedit

It is important to note that JSON objects (delimited by {} and typically associated with maps) are unordered, in other words, they do not maintain order. JSON arrays (typically associated with lists or sequences) are ordered, that is, they do preserve the initial insertion order. This impacts the way objects are read from Elasticsearch as one might find the insertion structure to be different than the extraction one. It is however easy to circumvent this problem - as JSON objects (maps) contain fields, use the field names (or keys) instead of their position inside the document to reliably get their values (in Java terms think of a JSON object as a HashMap as oppose to a LinkedHashMap).

Geo typesedit

For geolocation Elasticsearch provides two dedicated types, namely geo_point and geo_shape which do not have a direct equivalent in any of the libraries elasticsearch-hadoop integrates with. Further more, Elasticsearch accepts multiple formats for each type (as there are different ways to represent data), in fact there are 4 different representations for geo_point and 9 for geo_shape. To go around this, the connector breaks down the geo types into primitives depending on the actual format used for their respective types.

For strongly-typed libraries (like SparkSQL DataFrames), the format needs to be known before hand and thus, elasticsearch-hadoop will sample the data asking elasticsearch-hadoop for one random document that is representative of the mapping, parse it and based on the values found, identify the format used and create the necessary schema. This happens automatically at start-up without any user interference. As always, the user data must all be using the same format (a requirement from SparkSQL) otherwise reading a different format will trigger an exception.

Note that typically handling of these types poses no issues for the user whether reading or writing data.

Handling array/multi-values fieldsedit

Elasticsearch treats fields with single or multi-values the same; in fact, the mapping provides no information about this. As a client, it means one cannot tell whether a field is single-valued or not until is actually being read. In most cases this is not an issue and elasticsearch-hadoop automatically creates the necessary list/array on the fly. However in environments with strict schema such as Spark SQL, changing a field actual value from its declared type is not allowed. Worse yet, this information needs to be available even before reading the data. Since the mapping is not conclusive enough, elasticsearch-hadoop allows the user to specify the extra information through field information, specifically es.read.field.as.array.include and es.read.field.as.array.exclude.

Automatic mappingedit

By default, Elasticsearch provides automatic index and mapping when data is added under an index that has not been created before. In other words, data can be added into Elasticsearch without the index and the mappings being defined a priori. This is quite convenient since Elasticsearch automatically adapts to the data being fed to it - moreover, if certain entries have extra fields, Elasticsearch schema-less nature allows them to be indexed without any issues.

It is important to remember that automatic mapping uses the payload values to identify its type, using the first document creates the mapping. elasticsearch-hadoop communicates with Elasticsearch through JSON which does not provide any type information, rather only the field names and their values. One can think of it as type erasure or information loss; for example JSON does not differentiate integer numeric types - byte, short, int, long are all placed in the same long bucket. this can have unexpected side-effects since the type information is guessed such as:

numbers mapped only as long/doubleedit

Whenever Elasticsearch encounters a number, it will allocate the largest type for it since it does not know the exact number type of the field. Allocating a small type (such as byte, int or float) can lead to problems if a future document is larger, so Elasticsearch uses a safe default. For example, the document:

{
    "tweet" {
        "user" : "kimchy",
        "message" : "This is a tweet!",
        "postDate" : "2009-11-15T14:12:12",
        "priority" : 4,
        "rank" : 12.3
    }
}

triggers the following mapping:

{ "test" : {
    "mappings" : {
      "index" : {
        "properties" : {
          "message" : {
            "type" : "string"
          },
          "postDate" : {
            "type" : "date",
            "format" : "dateOptionalTime"      
          },
          "priority" : {
            "type" : "long"                    
          },
          "rank" : {
            "type" : "double"                  
          },
          "user" : {
            "type" : "string"
          }
        }
      }
    }
  }
}

The postDate field was recognized as a date in ISO 8601 format (dateOptionalTime)

The integer number (4) was mapped to the largest available type (long)

The fractional number (12.3) was mapped to the largest available type (double)

incorrect mappingedit

This happens when a string field contains numbers (say 1234) - Elasticsearch has no information that the number is actually a string and thus it map the field as a number, causing a parsing exception when a string is encountered. For example, this document:

{ "array":[123, "string"] }

causes an exception with automatic mapping:

{"error":"MapperParsingException[failed to parse [array]]; nested: NumberFormatException[For input string: \"string\"]; ","status":400}

because the field array is initially detected as a number (because of 123) which causes "string" to trigger the parsing exception since clearly it is not a number. The same issue tends to occur with strings might be interpreted as dates.

Hence if the defaults need to be overridden and/or if you experience the problems exposed above, potentially due to a diverse dataset, consider using Explicit mapping.

Disabling automatic mappingedit

Elasticsearch allows automatic index creation as well as dynamic mapping (for extra fields present in documents) to be disabled through the action.auto_create_index and index.mapper.dynamic settings on the nodes config files. As a safety net, elasticsearch-hadoop provides a dedicated configuration option es.index.auto.create which allows elasticsearch-hadoop to either create the index or not without having to modify the Elasticsearch cluster options.

Explicit mappingedit

Explicit or manual mapping should be considered when the defaults need to be overridden, if the data is detected incorrectly (as explained above) or, in most cases, to customize the index analysis. Refer to Elasticsearch create index and mapping documentation on how to define an index and its types - note that these need to be present before data is being uploaded to Elasticsearch (otherwise automatic mapping will be used by Elasticsearch, if enabled).

In most cases, templates are quite handy as they are automatically applied to new indices created that match the pattern; in other words instead of defining the mapping per index, one can just define the template once and then have it applied to all indices that match its pattern.