Skip to content

Commit 6222363

Browse files
committed
[SPARK] Improve array handling in SQL
Refactor logic to use absolute field names as the relative name are prone to overwrites Improve field filtering to allow partial or strict matching relates #482 #484
1 parent 53e10a3 commit 6222363

File tree

12 files changed

+240
-68
lines changed

12 files changed

+240
-68
lines changed

mr/src/itest/java/org/elasticsearch/hadoop/integration/mr/AbstractMROldApiSaveTest.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -61,7 +61,7 @@
6161
import org.junit.runners.Parameterized;
6262
import org.junit.runners.Parameterized.Parameters;
6363

64-
import static org.junit.Assume.assumeFalse;
64+
import static org.junit.Assume.*;
6565

6666
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
6767
@RunWith(Parameterized.class)

mr/src/main/java/org/elasticsearch/hadoop/serialization/Parser.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -107,6 +107,8 @@ enum NumberType {
107107

108108
void skipChildren();
109109

110+
String absoluteName();
111+
110112
String currentName();
111113

112114
Object currentValue();
@@ -133,6 +135,7 @@ enum NumberType {
133135

134136
byte[] binaryValue();
135137

138+
@Override
136139
void close();
137140

138141
// Fairly experimental methods

mr/src/main/java/org/elasticsearch/hadoop/serialization/dto/mapping/MappingUtils.java

Lines changed: 23 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@
3030
import org.apache.commons.logging.Log;
3131
import org.elasticsearch.hadoop.EsHadoopIllegalArgumentException;
3232
import org.elasticsearch.hadoop.cfg.FieldPresenceValidation;
33+
import org.elasticsearch.hadoop.serialization.FieldType;
3334
import org.elasticsearch.hadoop.serialization.field.FieldFilter;
3435
import org.elasticsearch.hadoop.util.StringUtils;
3536

@@ -126,16 +127,28 @@ private static String removeDoubleBrackets(List col) {
126127
}
127128
return col.toString();
128129
}
129-
130+
130131
public static Field filter(Field field, Collection<String> includes, Collection<String> excludes) {
131-
List<Field> filtered = new ArrayList<Field>();
132-
133-
for (Field fl : field.properties()) {
134-
if (FieldFilter.filter(fl.name(), includes, excludes)) {
135-
filtered.add(fl);
136-
}
137-
}
138-
139-
return new Field(field.name(), field.type(), filtered);
132+
List<Field> filtered = new ArrayList<Field>();
133+
134+
for (Field fl : field.skipHeaders().properties()) {
135+
processField(fl, null, filtered, includes, excludes);
136+
}
137+
138+
return new Field(field.name(), field.type(), filtered);
139+
}
140+
141+
private static void processField(Field field, String parentName, List<Field> filtered, Collection<String> includes, Collection<String> excludes) {
142+
String fieldName = (parentName != null ? parentName + "." + field.name() : field.name());
143+
144+
if (FieldFilter.filter(fieldName, includes, excludes)) {
145+
filtered.add(field);
146+
}
147+
148+
if (FieldType.OBJECT == field.type()) {
149+
for (Field nestedField : field.properties()) {
150+
processField(nestedField, fieldName, filtered, includes, excludes);
151+
}
152+
}
140153
}
141154
}

mr/src/main/java/org/elasticsearch/hadoop/serialization/field/FieldFilter.java

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,7 @@ public abstract class FieldFilter {
3333
* @param excludes
3434
* @return
3535
*/
36-
public static boolean filter(String path, Collection<String> includes, Collection<String> excludes) {
36+
public static boolean filter(String path, Collection<String> includes, Collection<String> excludes, boolean allowPartialMatches) {
3737
includes = (includes == null ? Collections.<String> emptyList() : includes);
3838
excludes = (excludes == null ? Collections.<String> emptyList() : excludes);
3939

@@ -81,11 +81,15 @@ else if (include.length() > path.length() && include.charAt(path.length()) == '.
8181
}
8282
}
8383

84-
if (pathIsPrefixOfAnInclude || exactIncludeMatch) {
85-
// if match or part of the path
84+
// if match or part of the path (based on the passed param)
85+
if (exactIncludeMatch || (allowPartialMatches && pathIsPrefixOfAnInclude)) {
8686
return true;
8787
}
88-
88+
8989
return false;
9090
}
91+
92+
public static boolean filter(String path, Collection<String> includes, Collection<String> excludes) {
93+
return filter(path, includes, excludes, true);
94+
}
9195
}

mr/src/main/java/org/elasticsearch/hadoop/serialization/json/JacksonJsonParser.java

Lines changed: 22 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,9 +20,12 @@
2020

2121
import java.io.IOException;
2222
import java.io.InputStream;
23+
import java.util.ArrayList;
24+
import java.util.List;
2325

2426
import org.codehaus.jackson.JsonFactory;
2527
import org.codehaus.jackson.JsonParser;
28+
import org.codehaus.jackson.JsonStreamContext;
2629
import org.codehaus.jackson.JsonToken;
2730
import org.codehaus.jackson.impl.JsonParserBase;
2831
import org.elasticsearch.hadoop.serialization.EsHadoopSerializationException;
@@ -135,6 +138,25 @@ public void skipChildren() {
135138
}
136139
}
137140

141+
@Override
142+
public String absoluteName() {
143+
List<String> tree = new ArrayList<String>();
144+
for (JsonStreamContext ctx = parser.getParsingContext(); ctx != null; ctx = ctx.getParent()) {
145+
if (ctx.inObject()) {
146+
tree.add(ctx.getCurrentName());
147+
}
148+
}
149+
StringBuilder sb = new StringBuilder();
150+
for (int index = tree.size(); index > 0; index--) {
151+
sb.append(tree.get(index - 1));
152+
sb.append(".");
153+
}
154+
155+
// remove the last .
156+
sb.setLength(sb.length() - 1);
157+
return sb.toString();
158+
}
159+
138160
@Override
139161
public String currentName() {
140162
try {

spark/sql-13/src/itest/scala/org/elasticsearch/spark/integration/AbstractScalaEsSparkSQL.scala

Lines changed: 98 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -22,13 +22,11 @@ import java.{lang => jl}
2222
import java.sql.Timestamp
2323
import java.{util => ju}
2424
import java.util.concurrent.TimeUnit
25-
import javax.xml.bind.DatatypeConverter
26-
2725
import scala.collection.JavaConversions.propertiesAsScalaMap
2826
import scala.collection.JavaConverters.asScalaBufferConverter
2927
import scala.collection.JavaConverters.mapAsJavaMapConverter
28+
import scala.collection.Map
3029
import scala.collection.mutable.ArrayBuffer
31-
3230
import org.apache.spark.SparkConf
3331
import org.apache.spark.SparkContext
3432
import org.apache.spark.SparkException
@@ -42,15 +40,14 @@ import org.apache.spark.sql.types.StructField
4240
import org.apache.spark.sql.types.StructType
4341
import org.apache.spark.sql.types.TimestampType
4442
import org.apache.spark.storage.StorageLevel._
45-
46-
import org.elasticsearch.hadoop.EsHadoopIllegalArgumentException
47-
import org.elasticsearch.hadoop.EsHadoopIllegalStateException
4843
import org.elasticsearch.hadoop.cfg.ConfigurationOptions._
44+
import org.elasticsearch.hadoop.cfg.PropertiesSettings
4945
import org.elasticsearch.hadoop.mr.RestUtils
5046
import org.elasticsearch.hadoop.util.StringUtils
5147
import org.elasticsearch.hadoop.util.TestSettings
5248
import org.elasticsearch.hadoop.util.TestUtils
5349
import org.elasticsearch.spark._
50+
import org.elasticsearch.spark.cfg._
5451
import org.elasticsearch.spark.sql._
5552
import org.elasticsearch.spark.sql.api.java.JavaEsSparkSQL
5653
import org.elasticsearch.spark.sql.sqlContextFunctions
@@ -64,11 +61,13 @@ import org.junit.BeforeClass
6461
import org.junit.FixMethodOrder
6562
import org.junit.Test
6663
import org.junit.runner.RunWith
67-
import org.junit.runners.MethodSorters
6864
import org.junit.runners.Parameterized
6965
import org.junit.runners.Parameterized.Parameters
66+
import org.junit.runners.MethodSorters
7067
import com.esotericsoftware.kryo.io.{Input => KryoInput}
7168
import com.esotericsoftware.kryo.io.{Output => KryoOutput}
69+
import javax.xml.bind.DatatypeConverter
70+
import org.elasticsearch.hadoop.EsHadoopIllegalArgumentException
7271

7372
object AbstractScalaEsScalaSparkSQL {
7473
@transient val conf = new SparkConf().set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
@@ -170,8 +169,8 @@ class AbstractScalaEsScalaSparkSQL(prefix: String, readMetadata: jl.Boolean, pus
170169
}
171170

172171
@Test
173-
def testArrayMapping() {
174-
val mapping = """{ "array-mapping": {
172+
def testArrayMappingFirstLevel() {
173+
val mapping = """{ "array-mapping-top-level": {
175174
| "properties" : {
176175
| "arr" : {
177176
| "properties" : {
@@ -185,7 +184,7 @@ class AbstractScalaEsScalaSparkSQL(prefix: String, readMetadata: jl.Boolean, pus
185184
}""".stripMargin
186185

187186
val index = wrapIndex("sparksql-test")
188-
val indexAndType = s"$index/array-mapping"
187+
val indexAndType = s"$index/array-mapping-top-level"
189188
RestUtils.touch(index)
190189
RestUtils.putMapping(indexAndType, mapping.getBytes(StringUtils.UTF_8))
191190

@@ -203,6 +202,87 @@ class AbstractScalaEsScalaSparkSQL(prefix: String, readMetadata: jl.Boolean, pus
203202
assertEquals(1, df.count())
204203
}
205204

205+
@Test
206+
def testMultiFieldsWithSameName {
207+
val index = wrapIndex("sparksql-test")
208+
val indexAndType = s"$index/array-mapping-nested"
209+
RestUtils.touch(index)
210+
211+
// add some data
212+
val jsonDoc = """{
213+
| "bar" : {
214+
| "bar" : {
215+
| "bar" : [{
216+
| "bar" : 1
217+
| }, {
218+
| "bar" : 2
219+
| }
220+
| ],
221+
| "level" : 2,
222+
| "level3" : true
223+
| },
224+
| "foo" : 10,
225+
| "level" : 1,
226+
| "level2" : 2
227+
| },
228+
| "foo" : "text",
229+
| "level" : 0,
230+
| "level1" : "string"
231+
|}
232+
""".stripMargin
233+
RestUtils.postData(indexAndType, jsonDoc.getBytes(StringUtils.UTF_8))
234+
RestUtils.refresh(index)
235+
236+
val newCfg = collection.mutable.Map(cfg.toSeq: _*) += ("es.field.read.as.array.include" -> "bar.bar.bar", "es.resource" -> indexAndType)
237+
val cfgSettings = new SparkSettingsManager().load(sc.getConf).copy().merge(newCfg.asJava)
238+
val schema = SchemaUtilsTestable.discoverMapping(cfgSettings)
239+
val mapping = SchemaUtilsTestable.rowInfo(cfgSettings)
240+
241+
val df = sqc.read.options(newCfg).format("org.elasticsearch.spark.sql").load(indexAndType)
242+
df.printSchema()
243+
df.take(1).foreach(println)
244+
assertEquals(1, df.count())
245+
}
246+
247+
@Test
248+
def testNestedFieldArray {
249+
val index = wrapIndex("sparksql-test")
250+
val indexAndType = s"$index/nested-same-name-fields"
251+
RestUtils.touch(index)
252+
253+
// add some data
254+
val jsonDoc = """{"foo" : 5, "nested": { "bar" : [{"date":"2015-01-01", "age":20},{"date":"2015-01-01", "age":20}], "what": "now" } }"""
255+
sc.makeRDD(Seq(jsonDoc)).saveJsonToEs(indexAndType)
256+
RestUtils.refresh(index)
257+
258+
val newCfg = collection.mutable.Map(cfg.toSeq: _*) += ("es.field.read.as.array.include" -> "nested.bar")
259+
260+
val df = sqc.read.options(newCfg).format("org.elasticsearch.spark.sql").load(indexAndType)
261+
df.printSchema()
262+
df.take(1).foreach(println)
263+
assertEquals(1, df.count())
264+
}
265+
266+
@Test
267+
def testArrayValue {
268+
val index = wrapIndex("sparksql-test")
269+
val indexAndType = s"$index/array-value"
270+
RestUtils.touch(index)
271+
272+
// add some data
273+
val jsonDoc = """{"array" : [1, 2, 4, 5] }"""
274+
sc.makeRDD(Seq(jsonDoc)).saveJsonToEs(indexAndType)
275+
RestUtils.refresh(index)
276+
277+
val newCfg = collection.mutable.Map(cfg.toSeq: _*) += ("es.field.read.as.array.include" -> "array")
278+
279+
val df = sqc.read.options(newCfg).format("org.elasticsearch.spark.sql").load(indexAndType)
280+
df.printSchema()
281+
df.take(1).foreach(println)
282+
assertEquals(1, df.count())
283+
}
284+
285+
206286
@Test
207287
def testBasicRead() {
208288
val dataFrame = artistsAsDataFrame
@@ -224,6 +304,14 @@ class AbstractScalaEsScalaSparkSQL(prefix: String, readMetadata: jl.Boolean, pus
224304
assertThat(RestUtils.get(target + "/_search?"), containsString("345"))
225305
}
226306

307+
@Test
308+
def testEsDataFrame1WriteCount() {
309+
val target = wrapIndex("sparksql-test/scala-basic-write")
310+
311+
val dataFrame = sqc.esDF(target, cfg)
312+
assertEquals(345, dataFrame.count())
313+
}
314+
227315
@Test
228316
def testEsDataFrame1WriteWithMapping() {
229317
val dataFrame = artistsAsDataFrame
Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,14 @@
1+
package org.elasticsearch.spark.sql
2+
3+
import org.elasticsearch.hadoop.cfg.Settings
4+
5+
object SchemaUtilsTestable {
6+
7+
def discoverMapping(cfg: Settings) = SchemaUtils.discoverMapping(cfg)
8+
9+
def rowInfo(cfg: Settings) = {
10+
val schema = SchemaUtils.discoverMapping(cfg)
11+
SchemaUtils.setRowInfo(cfg, schema.struct)
12+
SchemaUtils.getRowInfo(cfg)
13+
}
14+
}

spark/sql-13/src/main/scala/org/elasticsearch/spark/sql/DefaultSource.scala

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -4,7 +4,9 @@ import java.util.Locale
44
import scala.None
55
import scala.Null
66
import scala.collection.JavaConverters.mapAsJavaMapConverter
7+
import scala.collection.mutable.ArrayOps
78
import scala.collection.mutable.LinkedHashMap
9+
import scala.collection.mutable.LinkedHashSet
810
import org.apache.spark.rdd.RDD
911
import org.apache.spark.sql.DataFrame
1012
import org.apache.spark.sql.Row
@@ -46,9 +48,7 @@ import org.elasticsearch.hadoop.util.IOUtils
4648
import org.elasticsearch.hadoop.util.StringUtils
4749
import org.elasticsearch.spark.cfg.SparkSettingsManager
4850
import org.elasticsearch.spark.serialization.ScalaValueWriter
49-
import scala.collection.mutable.ListBuffer
50-
import scala.collection.mutable.LinkedHashSet
51-
import scala.collection.mutable.ArrayOps
51+
import org.apache.commons.logging.LogFactory
5252

5353
private[sql] class DefaultSource extends RelationProvider with SchemaRelationProvider with CreatableRelationProvider {
5454

@@ -122,7 +122,7 @@ private[sql] case class ElasticsearchRelation(parameters: Map[String, String], @
122122
}
123123

124124
if (filters != null && filters.size > 0 && Utils.isPushDown(cfg)) {
125-
val log = Utils.logger("org.elasticsearch.spark.sql.DataSource")
125+
val log = logger
126126
if (log.isDebugEnabled()) {
127127
log.debug(s"Pushing down filters ${filters.mkString("[", ",", "]")}")
128128
}
@@ -297,7 +297,7 @@ private[sql] case class ElasticsearchRelation(parameters: Map[String, String], @
297297

298298
def insert(data: DataFrame, overwrite: Boolean) {
299299
if (overwrite) {
300-
Utils.logger("org.elasticsearch.spark.sql.DataSource").info(s"Overwriting data for ${cfg.getResourceWrite}")
300+
logger.info(s"Overwriting data for ${cfg.getResourceWrite}")
301301

302302
// perform a scan-scroll delete
303303
val cfgCopy = cfg.copy()
@@ -318,4 +318,6 @@ private[sql] case class ElasticsearchRelation(parameters: Map[String, String], @
318318
rr.close()
319319
empty
320320
}
321+
322+
private def logger = LogFactory.getLog("org.elasticsearch.spark.sql.DataSource")
321323
}

0 commit comments

Comments
 (0)