[ 
https://issues.apache.org/jira/browse/DRILL-4303?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16955626#comment-16955626
 ] 

ASF GitHub Bot commented on DRILL-4303:
---------------------------------------

paul-rogers commented on pull request #1858: DRILL-4303: ESRI Shapefile (shp) 
Format Plugin
URL: https://github.com/apache/drill/pull/1858#discussion_r336796751
 
 

 ##########
 File path: 
contrib/format-esri/src/main/java/org/apache/drill/exec/store/esri/ShpBatchReader.java
 ##########
 @@ -0,0 +1,334 @@
+/*
+ * 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.esri;
+
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.GeometryCursor;
+import com.esri.core.geometry.ShapefileReader;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+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.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.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
+import org.jamel.dbf.DbfReader;
+import org.jamel.dbf.structure.DbfField;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class ShpBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private FileSplit split;
+  private BufferedReader reader;
+  private ResultSetLoader loader;
+  private ShpReaderConfig readerConfig;
+  private Path hadoopShp;
+  private Path hadoopDbf;
+  private Path hadoopPrj;
+  private FSDataInputStream fileReaderShp = null;
+  private FSDataInputStream fileReaderDbf = null;
+  private FSDataInputStream fileReaderPrj = null;
+  private GeometryCursor geomCursor = null;
+  private DbfReader dbfReader = null;
+  private ScalarWriter gidWriter;
+  private ScalarWriter sridWriter;
+  private ScalarWriter shapeTypeWriter;
+  private ScalarWriter geomWriter;
+  private RowSetLoader rowWriter;
+
+
+  private int srid;
+  private SpatialReference spatialReference;
+  private static final Logger logger = 
LoggerFactory.getLogger(ShpBatchReader.class);
+
+  public static class ShpReaderConfig {
+    protected final ShpFormatPlugin plugin;
+
+    public ShpReaderConfig(ShpFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public ShpBatchReader(ShpReaderConfig readerConfig) {
+    this.readerConfig = readerConfig;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    this.split = negotiator.split();
+    this.hadoopShp = split.getPath();
+    this.hadoopDbf = new Path(split.getPath().toString().replace("shp", 
"dbf"));
+    this.hadoopPrj = new Path(split.getPath().toString().replace("shp", 
"prj"));
+
+    openFile(negotiator);
+    SchemaBuilder builder = new SchemaBuilder();
+    builder.addNullable("gid", TypeProtos.MinorType.INT);
+    builder.addNullable("srid", TypeProtos.MinorType.INT);
+    builder.addNullable("shapeType", TypeProtos.MinorType.VARCHAR);
+    builder.addNullable("geom", TypeProtos.MinorType.VARBINARY);
+
+    negotiator.setTableSchema(builder.buildSchema(), false);
+    loader = negotiator.build();
+
+    rowWriter = loader.writer();
+    gidWriter = rowWriter.scalar("gid");
+    sridWriter = rowWriter.scalar("srid");
+    shapeTypeWriter = rowWriter.scalar("shapeType");
+    geomWriter = rowWriter.scalar("geom");
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    if (fileReaderShp == null) {
+      return false;
+    }
+
+    Geometry geom = null;
+
+    while( !rowWriter.isFull()){
+      Object[] dbfRow = dbfReader.nextRecord();
+      geom = geomCursor.next();
+      if(geom == null) {
+        return false;
+      }
+      processShapefileSet(rowWriter, geomCursor.getGeometryID(), geom, dbfRow);
+    }
+    return true;
+  }
+
+  private void openFile(FileSchemaNegotiator negotiator) {
+    try {
+      fileReaderShp = negotiator.fileSystem().open(split.getPath());
+      byte[] shpBuf = new byte[fileReaderShp.available()];
+      fileReaderShp.readFully(shpBuf);
+
+      ByteBuffer byteBuffer = ByteBuffer.wrap(shpBuf);
+      byteBuffer.position(byteBuffer.position() + 100);
+
+      ShapefileReader shpReader = new ShapefileReader();
+      geomCursor = shpReader.getGeometryCursor(byteBuffer);
+
+      fileReaderDbf = negotiator.fileSystem().open(hadoopDbf);
+      dbfReader = new DbfReader(fileReaderDbf);
+
+      fileReaderPrj = negotiator.fileSystem().open(hadoopPrj);
+      byte[] prjBuf = new byte[fileReaderPrj.available()];
+      fileReaderPrj.readFully(prjBuf);
+      fileReaderPrj.close();
+
+      String wktReference = new String(prjBuf);
+
+      String sridPatternText = 
"AUTHORITY\\[\"\\w+\"\\s*,\\s*\"*(\\d+)\"*\\]\\]$";
+      Pattern pattern = Pattern.compile(sridPatternText);
+      Matcher matcher = pattern.matcher(wktReference);
+      if (matcher.find()) {
+        this.srid = Integer.parseInt(matcher.group(1));
+        this.spatialReference = SpatialReference.create(srid);
+      }
+
+      logger.debug("Processing Shape File : {}", hadoopShp);
+    } catch (IOException e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath())
+        .addContext("User name", negotiator.userName())
+        .build(logger);
+    }
+  }
+
+  private String byteArrayToString(byte[] in) {
+    char out[] = new char[in.length * 2];
+    for (int i = 0; i < in.length; i++) {
+      out[i * 2] = "0123456789ABCDEF".charAt((in[i] >> 4) & 15);
+      out[i * 2 + 1] = "0123456789ABCDEF".charAt(in[i] & 15);
+    }
+    return new String(out);
+  }
+
+  private void processShapefileSet(RowSetLoader rowWriter, final int gid, 
final Geometry geom, final Object[] dbfRow) {
+    rowWriter.start();
+
+    gidWriter.setInt(gid);
+    sridWriter.setInt(srid);
+    shapeTypeWriter.setString(geom.getType().toString());
+    final ByteBuffer buf = (ByteBuffer) 
OGCGeometry.createFromEsriGeometry(geom, spatialReference).asBinary();
+    final byte[] bytes = buf.array();
+    geomWriter.setBytes(bytes, bytes.length);
+    writeDbfRow(dbfRow, rowWriter);
+
+    rowWriter.save();
+  }
+
+  private void writeDbfRow(final Object[] dbfRow, RowSetLoader rowWriter) {
+    int dbfFieldCount = dbfReader.getHeader().getFieldsCount();
+
+    for (int i = 0; i < dbfFieldCount; i++) {
+      DbfField field = dbfReader.getHeader().getField(i);
+
+      if (dbfRow[i] == null) {
+        continue;
+      }
+
+      switch (field.getDataType()) {
 
 Review comment:
   As suggested in another plugin, if fields are in a fixed order, would be 
faster to create classes for each type, then dispatch to the correct function. 
Index into an array for the column object. See example in that other review.
 
----------------------------------------------------------------
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:
us...@infra.apache.org


> ESRI Shapefile (shp) format plugin
> ----------------------------------
>
>                 Key: DRILL-4303
>                 URL: https://issues.apache.org/jira/browse/DRILL-4303
>             Project: Apache Drill
>          Issue Type: Improvement
>          Components: Storage - Other
>    Affects Versions: 1.17.0
>            Reporter: Karol Potocki
>            Assignee: Charles Givre
>            Priority: Major
>              Labels: doc-impacting
>             Fix For: 1.17.0
>
>
> Allow Drill (drill-gis) to read esri shapefiles, one of the most popular 
> geospatial formats.
> Format described here:
> https://www.esri.com/library/whitepapers/pdfs/shapefile.pdf
> It consists of three files (prj - srid information, dbf - data fields, shp - 
> geometry data)



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to