You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2020/12/18 18:30:00 UTC

[GitHub] [drill] vvysotskyi commented on a change in pull request #2129: DRILL-7823 - Add XML Format Plugin

vvysotskyi commented on a change in pull request #2129:
URL: https://github.com/apache/drill/pull/2129#discussion_r545995037



##########
File path: contrib/format-xml/src/main/java/org/apache/drill/exec/store/xml/XMLBatchReader.java
##########
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.drill.exec.store.xml;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.InputStream;
+
+
+public class XMLBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(XMLBatchReader.class);
+
+
+  private FileSplit split;
+  private InputStream fsStream;

Review comment:
       This field is used only in the `openFile() method, is it required to be a class field?

##########
File path: contrib/format-xml/src/main/java/org/apache/drill/exec/store/xml/XMLFormatConfig.java
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.drill.exec.store.xml;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+@JsonTypeName(XMLFormatPlugin.DEFAULT_NAME)
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class XMLFormatConfig implements FormatPluginConfig {
+
+  public final List<String> extensions;
+  public final int dataLevel;
+
+  public XMLBatchReader.XMLReaderConfig getReaderConfig(XMLFormatPlugin plugin) {

Review comment:
       Please move this method below the constructor.

##########
File path: contrib/format-xml/src/main/java/org/apache/drill/exec/store/xml/XMLMap.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.drill.exec.store.xml;
+
+import org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+
+import java.util.Objects;
+
+public class XMLMap {
+
+  private final String mapName;
+  private final int nestingLevel;
+  private final TupleWriter mapWriter;
+
+  public XMLMap (String mapName, int nestingLevel, TupleWriter mapWriter) {
+    this.mapName = mapName;
+    this.nestingLevel = nestingLevel;
+    this.mapWriter = mapWriter;
+  }
+
+  public TupleWriter getMapWriter() {
+    return mapWriter;
+  }
+
+  @Override
+  public boolean equals(Object obj) {

Review comment:
       Please don't forget adding `hashCode()` method every time when adding `equals()` one.

##########
File path: contrib/format-xml/src/main/java/org/apache/drill/exec/store/xml/XMLMap.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.drill.exec.store.xml;
+
+import org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+
+import java.util.Objects;
+
+public class XMLMap {
+
+  private final String mapName;
+  private final int nestingLevel;

Review comment:
       Could you please explain the goal for this field? Currently, it is used only in `equals()` and `toString()` methods.

##########
File path: contrib/format-xml/src/main/java/org/apache/drill/exec/store/xml/XMLUtils.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.drill.exec.store.xml;
+
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.xml.stream.XMLStreamConstants;
+import javax.xml.stream.events.XMLEvent;
+
+public class XMLUtils {
+
+  private static final Logger logger = LoggerFactory.getLogger(XMLUtils.class);

Review comment:
       The logger is not used, please remove it.

##########
File path: contrib/format-xml/src/test/resources/logback-test.txt
##########
@@ -0,0 +1,69 @@
+<?xml version="1.0" encoding="UTF-8" ?>

Review comment:
       Please remove this file, it is not used.

##########
File path: contrib/format-xml/src/main/java/org/apache/drill/exec/store/xml/XMLReader.java
##########
@@ -0,0 +1,411 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.drill.exec.store.xml;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamConstants;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Stack;
+
+public class XMLReader {
+  private static final Logger logger = LoggerFactory.getLogger(XMLReader.class);
+  private final String ATTRIBUTE_MAP_NAME = "attributes";

Review comment:
       Looks like it should be static.

##########
File path: contrib/format-xml/README.md
##########
@@ -0,0 +1,76 @@
+# XML Format Reader
+This plugin enables Drill to read XML files without defining any kind of schema.
+
+## Configuration
+Aside from the file extension, there is one configuration option:
+
+* `dataLevel`: XML data often contains a considerable amount of nesting which is not necesarily useful for data analysis. This parameter allows you to set the nesting level 
+  where the data actually starts.  The levels start at `1`.
+
+The default configuration is shown below:
+
+```json
+"xml": {
+  "type": "xml",
+  "extensions": [
+    "xml"
+  ],
+  "dataLevel": 2
+}
+```
+
+## Data Types
+All fields are read as strings.  Nested fields are read as maps.  Future functionality could include support for lists.
+
+## Limitations: Schema Ambiguity
+XML is a challenging format to process as the structure does not give any hints about the schema.  For example, a JSON file might have the following record:
+
+```json
+"record" : {
+  "intField:" : 1,
+  "listField" : [1, 2],
+  "otherField" : {
+    "nestedField1" : "foo",
+    "nestedField2" : "bar"
+  }
+}
+```
+
+From this data, it is clear that `listField` is a `list` and `otherField` is a map.  This same data could be represented in XML as follows:
+
+```xml
+<record>
+  <intField>1</intField>

Review comment:
       Please also document attributes handling.

##########
File path: contrib/format-xml/src/main/java/org/apache/drill/exec/store/xml/XMLBatchReader.java
##########
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.drill.exec.store.xml;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.apache.hadoop.mapred.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.InputStream;
+
+
+public class XMLBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(XMLBatchReader.class);
+
+
+  private FileSplit split;
+  private InputStream fsStream;
+  private RowSetLoader rootRowWriter;
+  private CustomErrorContext errorContext;
+
+  private XMLReader reader;
+  private final int maxRecords;
+  private final int dataLevel;
+
+  private enum xmlState {

Review comment:
       Looks like this enum is unused. Could you please remove it?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org