Github user magpierre commented on a diff in the pull request:

    https://github.com/apache/drill/pull/451#discussion_r58943891
  
    --- Diff: 
contrib/storage-xml/src/main/java/org/apache/drill/exec/store/easy/xml/XMLRecordReader.java
 ---
    @@ -0,0 +1,95 @@
    +/**
    + * 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.easy.xml;
    +
    +import com.fasterxml.jackson.annotation.JsonInclude;
    +import com.fasterxml.jackson.databind.JsonNode;
    +import com.fasterxml.jackson.databind.ObjectMapper;
    +import org.apache.drill.common.exceptions.ExecutionSetupException;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.ops.OperatorContext;
    +import org.apache.drill.exec.physical.impl.OutputMutator;
    +import org.apache.drill.exec.store.dfs.DrillFileSystem;
    +import org.apache.drill.exec.store.easy.json.JSONRecordReader;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.Path;
    +import org.xml.sax.SAXException;
    +import javax.xml.parsers.ParserConfigurationException;
    +import javax.xml.parsers.SAXParser;
    +import javax.xml.parsers.SAXParserFactory;
    +import java.io.IOException;
    +import java.util.List;
    +
    +
    +public class XMLRecordReader extends JSONRecordReader {
    +    private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(XMLRecordReader.class);
    +    private XMLSaxParser handler;
    +    private SAXParser xmlParser;
    +    private JsonNode node;
    +
    +    public XMLRecordReader(FragmentContext fragmentContext, String 
inputPath, DrillFileSystem fileSystem, List<SchemaPath> columns, 
XMLFormatPlugin.XMLFormatConfig xmlConfig) throws OutOfMemoryException {
    +        super(fragmentContext, inputPath, fileSystem, columns);
    +        try {
    +            FSDataInputStream fsStream = fileSystem.open(new 
Path(inputPath));
    +            SAXParserFactory saxParserFactory = 
SAXParserFactory.newInstance();
    +            xmlParser = saxParserFactory.newSAXParser();
    +            handler = new XMLSaxParser();
    +            handler.setRemoveNameSpace(!xmlConfig.getKeepPrefix());
    +            xmlParser.parse(fsStream.getWrappedStream(), handler);
    +            ObjectMapper mapper = new ObjectMapper();
    +            node = mapper.valueToTree(handler.getVal());
    --- End diff --
    
    The XML parser is streaming based. Due to SAX which is a stateless parser 
where each tag triggers a series of events and then drops the information, it 
comes with a very little memory overhead. However, as noted I keep all parsed 
objects in a stack since I need to link them. Due to the way the parsing works 
the root of the document is really getting its children last, which means that 
once parsed I have the whole document in memory but in JSON format so I will at 
least not keep both a JSON document and an XML document in memory at the same 
time. I've been considering storing state in files instead but I don't think it 
would be performing well considering I need to revisit objects frequently. If 
you have any suggestions such as memory mapped files that could spill down to 
disk if large, it would be appreciated. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to