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

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

cgivre commented on pull request #1862: DRILL-7385: Convert PCAP Format Plugin 
to EVF
URL: https://github.com/apache/drill/pull/1862#discussion_r332305033
 
 

 ##########
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapBatchReader.java
 ##########
 @@ -0,0 +1,295 @@
+/*
+ * 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.pcap;
+
+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.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.pcap.decoder.Packet;
+import org.apache.drill.exec.store.pcap.decoder.PacketDecoder;
+import org.apache.drill.exec.store.pcap.schema.Schema;
+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.mapred.FileSplit;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import org.apache.hadoop.fs.Path;
+
+import static 
org.apache.drill.exec.store.pcap.PcapFormatUtils.parseBytesToASCII;
+
+public class PcapBatchReader implements ManagedReader<FileSchemaNegotiator> {
+
+  private FileSplit split;
+
+  private PcapReaderConfig readerConfig;
+
+  private PacketDecoder decoder;
+
+  private ResultSetLoader loader;
+
+  private FSDataInputStream fsStream;
+
+  private Schema pcapSchema;
+
+  private int validBytes;
+
+  private byte[] buffer;
+
+  private int offset = 0;
+
+  static final int BUFFER_SIZE = 500_000;
+
+  private static final Logger logger = 
LoggerFactory.getLogger(PcapBatchReader.class);
+
+  public static class PcapReaderConfig {
+
+    protected final PcapFormatPlugin plugin;
+    public PcapReaderConfig(PcapFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PcapBatchReader(PcapReaderConfig readerConfig) {
+    this.readerConfig = readerConfig;
+  }
+
+  @Override
+  public boolean open(FileSchemaNegotiator negotiator) {
+    split = negotiator.split();
+    openFile(negotiator);
+    SchemaBuilder builder = new SchemaBuilder();
+    this.pcapSchema = new Schema();
+    TupleMetadata schema = pcapSchema.buildSchema(builder);
+    negotiator.setTableSchema(schema, false);
+    this.loader = negotiator.build();
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    RowSetLoader rowWriter = loader.writer();
+    while (!rowWriter.isFull()) {
+      if (!parseNextPacket(rowWriter)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public void close() {
+    try {
+      fsStream.close();
+    } catch (IOException e) {
+      throw UserException.
+        dataReadError()
+        .addContext("Error closing InputStream: " + e.getMessage())
+        .build(logger);
+    }
+    fsStream = null;
+    this.buffer = null;
+    this.decoder = null;
+  }
+
+  private void openFile(FileSchemaNegotiator negotiator) {
+    String filePath = null;
+    try {
+      filePath = split.getPath().toString();
+      this.fsStream = negotiator.fileSystem().open(new Path(filePath));
+      this.decoder = new PacketDecoder(fsStream);
+      this.buffer = new byte[BUFFER_SIZE + decoder.getMaxLength()];
+      this.validBytes = fsStream.read(buffer);
+    } catch (IOException io) {
+      throw UserException.dataReadError(io).addContext("File name:", 
filePath).build(logger);
+    }
+  }
+
+  private boolean parseNextPacket(RowSetLoader rowWriter){
+    Packet packet = new Packet();
+
+    if(offset >= validBytes){
+      return false;
+    }
+    if (validBytes - offset < decoder.getMaxLength()) {
+      getNextPacket(rowWriter);
+    }
+
+    if(packet == null) {
+      return false;
+    }
+    int old = offset;
+    offset = decoder.decodePacket(buffer, offset, packet, 
decoder.getMaxLength(), validBytes);
+    if (offset > validBytes) {
+      logger.error("Invalid packet at offset {}", old);
+    }
+    addDataToTable(packet, decoder.getNetwork(), rowWriter);
+
+    return true;
+  }
+
+  private boolean getNextPacket(RowSetLoader rowWriter) {
+    Packet packet = new Packet();
+    try {
+      if (validBytes == buffer.length) {
+        // shift data and read more. This is the common case.
+        System.arraycopy(buffer, offset, buffer, 0, validBytes - offset);
+        validBytes = validBytes - offset;
+        offset = 0;
+
+        int n = fsStream.read(buffer, validBytes, buffer.length - validBytes);
+        if (n > 0) {
+          validBytes += n;
+        }
+        logger.info("read {} bytes, at {} offset", n, validBytes);
+      } else {
+        // near the end of the file, we will just top up the buffer without 
shifting
+        int n = fsStream.read(buffer, offset, buffer.length - offset);
+        if (n > 0) {
+          validBytes = validBytes + n;
+          logger.info("Topped up buffer with {} bytes to yield {}\n", n, 
validBytes);
+        }
+      }
+    } catch (Exception e) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean addDataToTable(Packet packet, int networkType,  RowSetLoader 
rowWriter) {
+    rowWriter.start();
+    writeStringColumn(rowWriter, "type", packet.getPacketType());
+    writeTimestampColumn(rowWriter, "timestamp", new 
Instant(packet.getTimestamp()));
+    writeLongValue(rowWriter, "timestamp_micro", packet.getTimestampMicro());
+    writeIntColumn(rowWriter, "network", networkType);
+    writeStringColumn(rowWriter, "src_mac_address", 
packet.getEthernetSource());
+    writeStringColumn(rowWriter, "dst_mac_address", 
packet.getEthernetDestination());
+
+    if (packet.getDst_ip() != null) {
+      writeStringColumn(rowWriter, "dst_ip", 
packet.getDst_ip().getHostAddress());
+    } else {
+      writeStringColumn(rowWriter, "dst_ip", null);
+    }
+    if (packet.getSrc_ip() != null) {
+      writeStringColumn(rowWriter, "src_ip", 
packet.getSrc_ip().getHostAddress());
+    } else {
+      writeStringColumn(rowWriter, "src_ip", null);
+    }
+    writeIntColumn(rowWriter, "src_port", packet.getSrc_port());
+    writeIntColumn(rowWriter, "dst_port", packet.getDst_port());
+    writeIntColumn(rowWriter, "packet_length", packet.getPacketLength());
+
+    // TCP Only Packet Data
+    if (packet.isTcpPacket()) {
+      writeLongValue(rowWriter, "tcp_session", packet.getSessionHash());
+      writeIntColumn(rowWriter, "tcp_sequence", packet.getSequenceNumber());
+      writeBooleanColumn(rowWriter, "tcp_ack", packet.getAckNumber());
+      writeIntColumn(rowWriter, "tcp_flags", packet.getFlags());
+      writeStringColumn(rowWriter, "tcp_parsed_flags", 
packet.getParsedFlags());
+      writeBooleanColumn(rowWriter, "tcp_flags_ns", (packet.getFlags() & 
0x100) != 0);
 
 Review comment:
   Would there be any performance benefit or just cleaner code?
 
----------------------------------------------------------------
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


> Convert PCAP Format Plugin to EVF
> ---------------------------------
>
>                 Key: DRILL-7385
>                 URL: https://issues.apache.org/jira/browse/DRILL-7385
>             Project: Apache Drill
>          Issue Type: Improvement
>    Affects Versions: 1.16.0
>            Reporter: Charles Givre
>            Assignee: Charles Givre
>            Priority: Major
>             Fix For: 1.17.0
>
>
> The PCAP format plugin has the possibility of extracting considerably more 
> data out of PCAP files.  To facilitate this, this PR updates the plugin to 
> use the new Enhanced Vector Framework.  No changes in functionality should 
> occur.



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

Reply via email to