[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-07-03 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/drill/pull/831


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-06-27 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r124427218
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapDrillTable.java
 ---
@@ -0,0 +1,73 @@
+/*
--- End diff --

The JIRA link is also good. Once the code is into Drill, it will be hard 
for future users to locate the corresponding JIRA. But, if you place a comment 
into a file somewhere, then future developers can find the JIRA.


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-06-27 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r124427466
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.AbstractRecordReader;
+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.dto.ColumnDto;
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+import org.apache.drill.exec.store.pcap.schema.Schema;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.drill.exec.store.pcap.Utils.parseBytesToASCII;
+
+public class PcapRecordReader extends AbstractRecordReader {
+
+  private OutputMutator output;
+
+  private final PacketDecoder decoder;
+  private ImmutableList projectedCols;
+
+  private byte[] buffer = new byte[10];
+  private int offset = 0;
+  private InputStream in;
+  private int validBytes;
+
+  private static final Map TYPES;
+
+  private static class ProjectedColumnInfo {
+ValueVector vv;
+ColumnDto pcapColumn;
+  }
+
+  static {
+TYPES = ImmutableMap.builder()
+.put(PcapTypes.STRING, MinorType.VARCHAR)
+.put(PcapTypes.INTEGER, MinorType.INT)
+.put(PcapTypes.LONG, MinorType.BIGINT)
+.put(PcapTypes.TIMESTAMP, MinorType.TIMESTAMP)
+.build();
+  }
+
+  public PcapRecordReader(final String inputPath,
+  final List projectedColumns) {
+try {
+  this.in = new FileInputStream(inputPath);
+  this.decoder = getPacketDecoder();
+  validBytes = in.read(buffer);
+} catch (IOException e) {
+  throw new RuntimeException("File " + inputPath + " not Found");
+}
+setColumns(projectedColumns);
+  }
+
+  @Override
+  public void setup(final OperatorContext context, final OutputMutator 
output) throws ExecutionSetupException {
+this.output = output;
+  }
+
+  @Override
+  public int next() {
+projectedCols = getProjectedColsIfItNull();
+try {
+  return parsePcapFilesAndPutItToTable();
+} catch (IOException io) {
+  throw new RuntimeException("Trouble with reading packets in file!");
+}
+  }
+
+  @Override
+  public void close() throws Exception {
+  }
+
+  private PacketDecoder getPacketDecoder() {
+try {
+  return new PacketDecoder(in);
+ 

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-06-27 Thread dmitriyHavrilovich
Github user dmitriyHavrilovich commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r124220475
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapDrillTable.java
 ---
@@ -0,0 +1,73 @@
+/*
--- End diff --

I think a link to a JIRA can be added. All needed information about already 
listed there.


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-06-27 Thread dmitriyHavrilovich
Github user dmitriyHavrilovich commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r124220049
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.AbstractRecordReader;
+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.dto.ColumnDto;
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+import org.apache.drill.exec.store.pcap.schema.Schema;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.drill.exec.store.pcap.Utils.parseBytesToASCII;
+
+public class PcapRecordReader extends AbstractRecordReader {
+
+  private OutputMutator output;
+
+  private final PacketDecoder decoder;
+  private ImmutableList projectedCols;
+
+  private byte[] buffer = new byte[10];
+  private int offset = 0;
+  private InputStream in;
+  private int validBytes;
+
+  private static final Map TYPES;
+
+  private static class ProjectedColumnInfo {
+ValueVector vv;
+ColumnDto pcapColumn;
+  }
+
+  static {
+TYPES = ImmutableMap.builder()
+.put(PcapTypes.STRING, MinorType.VARCHAR)
+.put(PcapTypes.INTEGER, MinorType.INT)
+.put(PcapTypes.LONG, MinorType.BIGINT)
+.put(PcapTypes.TIMESTAMP, MinorType.TIMESTAMP)
+.build();
+  }
+
+  public PcapRecordReader(final String inputPath,
+  final List projectedColumns) {
+try {
+  this.in = new FileInputStream(inputPath);
+  this.decoder = getPacketDecoder();
+  validBytes = in.read(buffer);
+} catch (IOException e) {
+  throw new RuntimeException("File " + inputPath + " not Found");
+}
+setColumns(projectedColumns);
+  }
+
+  @Override
+  public void setup(final OperatorContext context, final OutputMutator 
output) throws ExecutionSetupException {
+this.output = output;
+  }
+
+  @Override
+  public int next() {
+projectedCols = getProjectedColsIfItNull();
+try {
+  return parsePcapFilesAndPutItToTable();
+} catch (IOException io) {
+  throw new RuntimeException("Trouble with reading packets in file!");
+}
+  }
+
+  @Override
+  public void close() throws Exception {
+  }
+
+  private PacketDecoder getPacketDecoder() {
+try {
+  return new 

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-06-27 Thread dmitriyHavrilovich
Github user dmitriyHavrilovich commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r124219513
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/Utils.java ---
@@ -0,0 +1,65 @@
+/*
+ * 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 com.google.common.primitives.Ints;
+import com.google.common.primitives.Shorts;
+
+public class Utils {
+
+  public static int getIntFileOrder(boolean byteOrder, final byte[] buf, 
final int offset) {
+if (byteOrder) {
--- End diff --

Comment will be added here, explaining this case


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-06-27 Thread dmitriyHavrilovich
Github user dmitriyHavrilovich commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r124218971
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/Packet.java
 ---
@@ -0,0 +1,371 @@
+/*
+ * 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.decoder;
+
+import com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import static org.apache.drill.exec.store.pcap.Utils.convertInt;
+import static org.apache.drill.exec.store.pcap.Utils.convertShort;
+import static org.apache.drill.exec.store.pcap.Utils.getByte;
+import static org.apache.drill.exec.store.pcap.Utils.getIntFileOrder;
+import static org.apache.drill.exec.store.pcap.Utils.getShort;
+
+public class Packet {
--- End diff --

This java libraries uses JNI to work with existing libpicap c library. So 
they are just wrappers above. This plugin uses only java code to process pcap.


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-06-21 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r123397439
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.java
 ---
@@ -0,0 +1,307 @@
+/*
+ * 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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+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.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.AbstractRecordReader;
+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.dto.ColumnDto;
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+import org.apache.drill.exec.store.pcap.schema.Schema;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.drill.exec.store.pcap.Utils.parseBytesToASCII;
+
+public class PcapRecordReader extends AbstractRecordReader {
+  private static final Logger logger = 
LoggerFactory.getLogger(PcapRecordReader.class);
+
+  private static final int BATCH_SIZE = 40_000;
+
+  private OutputMutator output;
+
+  private PacketDecoder decoder;
+  private ImmutableList projectedCols;
+
+  private byte[] buffer;
+  private int offset = 0;
+  private InputStream in;
+  private int validBytes;
+
+  private String inputPath;
+  private List projectedColumns;
+
+  private static final Map TYPES;
+
+  private static class ProjectedColumnInfo {
+ValueVector vv;
+ColumnDto pcapColumn;
+  }
+
+  static {
+TYPES = ImmutableMap.builder()
+.put(PcapTypes.STRING, MinorType.VARCHAR)
+.put(PcapTypes.INTEGER, MinorType.INT)
+.put(PcapTypes.LONG, MinorType.BIGINT)
+.put(PcapTypes.TIMESTAMP, MinorType.TIMESTAMP)
+.build();
+  }
+
+  public PcapRecordReader(final String inputPath,
+  final List projectedColumns) {
+this.inputPath = inputPath;
+this.projectedColumns = projectedColumns;
+  }
+
+  @Override
+  public void setup(final OperatorContext context, final OutputMutator 
output) throws ExecutionSetupException {
+try {
+
+  this.output = output;
+  this.buffer = new byte[10];
+  this.in = new FileInputStream(inputPath);
+  this.decoder = new PacketDecoder(in);
+  this.validBytes = in.read(buffer);
+  this.projectedCols = getProjectedColsIfItNull();
+  setColumns(projectedColumns);
+} catch (IOException io) {
+  throw UserException.dataReadError(io)
   

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-25 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118619596
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/Utils.java ---
@@ -0,0 +1,65 @@
+/*
+ * 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 com.google.common.primitives.Ints;
+import com.google.common.primitives.Shorts;
+
+public class Utils {
+
+  public static int getIntFileOrder(boolean byteOrder, final byte[] buf, 
final int offset) {
+if (byteOrder) {
--- End diff --

Maybe an explanation of mapping byte order to booleans? true/false = 
which/which endian?


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-25 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118619502
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.AbstractRecordReader;
+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.dto.ColumnDto;
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+import org.apache.drill.exec.store.pcap.schema.Schema;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.drill.exec.store.pcap.Utils.parseBytesToASCII;
+
+public class PcapRecordReader extends AbstractRecordReader {
+
+  private OutputMutator output;
+
+  private final PacketDecoder decoder;
+  private ImmutableList projectedCols;
+
+  private byte[] buffer = new byte[10];
+  private int offset = 0;
+  private InputStream in;
+  private int validBytes;
+
+  private static final Map TYPES;
+
+  private static class ProjectedColumnInfo {
+ValueVector vv;
+ColumnDto pcapColumn;
+  }
+
+  static {
+TYPES = ImmutableMap.builder()
+.put(PcapTypes.STRING, MinorType.VARCHAR)
+.put(PcapTypes.INTEGER, MinorType.INT)
+.put(PcapTypes.LONG, MinorType.BIGINT)
+.put(PcapTypes.TIMESTAMP, MinorType.TIMESTAMP)
+.build();
+  }
+
+  public PcapRecordReader(final String inputPath,
+  final List projectedColumns) {
+try {
+  this.in = new FileInputStream(inputPath);
+  this.decoder = getPacketDecoder();
+  validBytes = in.read(buffer);
+} catch (IOException e) {
+  throw new RuntimeException("File " + inputPath + " not Found");
+}
+setColumns(projectedColumns);
+  }
+
+  @Override
+  public void setup(final OperatorContext context, final OutputMutator 
output) throws ExecutionSetupException {
+this.output = output;
+  }
+
+  @Override
+  public int next() {
+projectedCols = getProjectedColsIfItNull();
+try {
+  return parsePcapFilesAndPutItToTable();
+} catch (IOException io) {
+  throw new RuntimeException("Trouble with reading packets in file!");
+}
+  }
+
+  @Override
+  public void close() throws Exception {
+  }
+
+  private PacketDecoder getPacketDecoder() {
+try {
+  return new PacketDecoder(in);
+ 

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-25 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118616554
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.AbstractRecordReader;
+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.dto.ColumnDto;
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+import org.apache.drill.exec.store.pcap.schema.Schema;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.drill.exec.store.pcap.Utils.parseBytesToASCII;
+
+public class PcapRecordReader extends AbstractRecordReader {
+
+  private OutputMutator output;
+
+  private final PacketDecoder decoder;
+  private ImmutableList projectedCols;
+
+  private byte[] buffer = new byte[10];
+  private int offset = 0;
+  private InputStream in;
+  private int validBytes;
+
+  private static final Map TYPES;
+
+  private static class ProjectedColumnInfo {
+ValueVector vv;
+ColumnDto pcapColumn;
+  }
+
+  static {
+TYPES = ImmutableMap.builder()
+.put(PcapTypes.STRING, MinorType.VARCHAR)
+.put(PcapTypes.INTEGER, MinorType.INT)
+.put(PcapTypes.LONG, MinorType.BIGINT)
+.put(PcapTypes.TIMESTAMP, MinorType.TIMESTAMP)
+.build();
+  }
+
+  public PcapRecordReader(final String inputPath,
+  final List projectedColumns) {
+try {
+  this.in = new FileInputStream(inputPath);
+  this.decoder = getPacketDecoder();
+  validBytes = in.read(buffer);
+} catch (IOException e) {
+  throw new RuntimeException("File " + inputPath + " not Found");
+}
+setColumns(projectedColumns);
+  }
+
+  @Override
+  public void setup(final OperatorContext context, final OutputMutator 
output) throws ExecutionSetupException {
+this.output = output;
+  }
+
+  @Override
+  public int next() {
+projectedCols = getProjectedColsIfItNull();
+try {
+  return parsePcapFilesAndPutItToTable();
+} catch (IOException io) {
+  throw new RuntimeException("Trouble with reading packets in file!");
+}
+  }
+
+  @Override
+  public void close() throws Exception {
--- End diff --

Close your file handle? Set buffer to null to hasten its reclamation?

Note that 

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-25 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118614384
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapDrillTable.java
 ---
@@ -0,0 +1,73 @@
+/*
--- End diff --

Would be very helpful if this PR can include a package-info.java file to 
describe this work. For example, what is pcap? Links to good sources? What 
features of Drill does it use (push-downs)? Etc.


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-25 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118620276
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.AbstractRecordReader;
+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.dto.ColumnDto;
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+import org.apache.drill.exec.store.pcap.schema.Schema;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.drill.exec.store.pcap.Utils.parseBytesToASCII;
+
+public class PcapRecordReader extends AbstractRecordReader {
+
+  private OutputMutator output;
+
+  private final PacketDecoder decoder;
+  private ImmutableList projectedCols;
+
+  private byte[] buffer = new byte[10];
+  private int offset = 0;
+  private InputStream in;
+  private int validBytes;
+
+  private static final Map TYPES;
+
+  private static class ProjectedColumnInfo {
+ValueVector vv;
+ColumnDto pcapColumn;
+  }
+
+  static {
+TYPES = ImmutableMap.builder()
+.put(PcapTypes.STRING, MinorType.VARCHAR)
+.put(PcapTypes.INTEGER, MinorType.INT)
+.put(PcapTypes.LONG, MinorType.BIGINT)
+.put(PcapTypes.TIMESTAMP, MinorType.TIMESTAMP)
+.build();
+  }
+
+  public PcapRecordReader(final String inputPath,
+  final List projectedColumns) {
+try {
+  this.in = new FileInputStream(inputPath);
+  this.decoder = getPacketDecoder();
+  validBytes = in.read(buffer);
+} catch (IOException e) {
+  throw new RuntimeException("File " + inputPath + " not Found");
+}
+setColumns(projectedColumns);
+  }
+
+  @Override
+  public void setup(final OperatorContext context, final OutputMutator 
output) throws ExecutionSetupException {
+this.output = output;
+  }
+
+  @Override
+  public int next() {
+projectedCols = getProjectedColsIfItNull();
+try {
+  return parsePcapFilesAndPutItToTable();
+} catch (IOException io) {
+  throw new RuntimeException("Trouble with reading packets in file!");
+}
+  }
+
+  @Override
+  public void close() throws Exception {
+  }
+
+  private PacketDecoder getPacketDecoder() {
+try {
+  return new PacketDecoder(in);
+ 

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-25 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118617482
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.AbstractRecordReader;
+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.dto.ColumnDto;
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+import org.apache.drill.exec.store.pcap.schema.Schema;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.drill.exec.store.pcap.Utils.parseBytesToASCII;
+
+public class PcapRecordReader extends AbstractRecordReader {
+
+  private OutputMutator output;
+
+  private final PacketDecoder decoder;
+  private ImmutableList projectedCols;
+
+  private byte[] buffer = new byte[10];
+  private int offset = 0;
+  private InputStream in;
+  private int validBytes;
+
+  private static final Map TYPES;
+
+  private static class ProjectedColumnInfo {
+ValueVector vv;
+ColumnDto pcapColumn;
+  }
+
+  static {
+TYPES = ImmutableMap.builder()
+.put(PcapTypes.STRING, MinorType.VARCHAR)
+.put(PcapTypes.INTEGER, MinorType.INT)
+.put(PcapTypes.LONG, MinorType.BIGINT)
+.put(PcapTypes.TIMESTAMP, MinorType.TIMESTAMP)
+.build();
+  }
+
+  public PcapRecordReader(final String inputPath,
+  final List projectedColumns) {
+try {
+  this.in = new FileInputStream(inputPath);
+  this.decoder = getPacketDecoder();
+  validBytes = in.read(buffer);
+} catch (IOException e) {
+  throw new RuntimeException("File " + inputPath + " not Found");
+}
+setColumns(projectedColumns);
+  }
+
+  @Override
+  public void setup(final OperatorContext context, final OutputMutator 
output) throws ExecutionSetupException {
+this.output = output;
+  }
+
+  @Override
+  public int next() {
+projectedCols = getProjectedColsIfItNull();
+try {
+  return parsePcapFilesAndPutItToTable();
--- End diff --

Drill has certain protocols that are not entirely obvious, but that are 
needed here. Each call to `next()` produces a batch of rows. Drill requires 
that the batch contain no more than 64K rows. (Bad things happen if the batch 
has more.) So, the trick for a record reader is to read up to some 

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-25 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118616406
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.AbstractRecordReader;
+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.dto.ColumnDto;
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+import org.apache.drill.exec.store.pcap.schema.Schema;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.drill.exec.store.pcap.Utils.parseBytesToASCII;
+
+public class PcapRecordReader extends AbstractRecordReader {
+
+  private OutputMutator output;
+
+  private final PacketDecoder decoder;
+  private ImmutableList projectedCols;
+
+  private byte[] buffer = new byte[10];
+  private int offset = 0;
+  private InputStream in;
+  private int validBytes;
+
+  private static final Map TYPES;
+
+  private static class ProjectedColumnInfo {
+ValueVector vv;
+ColumnDto pcapColumn;
+  }
+
+  static {
+TYPES = ImmutableMap.builder()
+.put(PcapTypes.STRING, MinorType.VARCHAR)
+.put(PcapTypes.INTEGER, MinorType.INT)
+.put(PcapTypes.LONG, MinorType.BIGINT)
+.put(PcapTypes.TIMESTAMP, MinorType.TIMESTAMP)
+.build();
+  }
+
+  public PcapRecordReader(final String inputPath,
+  final List projectedColumns) {
+try {
+  this.in = new FileInputStream(inputPath);
--- End diff --

As noted above, by opening the file here, if you are scanning 1000 files, 
you'll have 1000 open file handles at the start of the fragment. Better to 
postpone opening files until setup.


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-25 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118617811
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.AbstractRecordReader;
+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.dto.ColumnDto;
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+import org.apache.drill.exec.store.pcap.schema.Schema;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.drill.exec.store.pcap.Utils.parseBytesToASCII;
+
+public class PcapRecordReader extends AbstractRecordReader {
+
+  private OutputMutator output;
+
+  private final PacketDecoder decoder;
+  private ImmutableList projectedCols;
+
+  private byte[] buffer = new byte[10];
+  private int offset = 0;
+  private InputStream in;
+  private int validBytes;
+
+  private static final Map TYPES;
+
+  private static class ProjectedColumnInfo {
+ValueVector vv;
+ColumnDto pcapColumn;
+  }
+
+  static {
+TYPES = ImmutableMap.builder()
+.put(PcapTypes.STRING, MinorType.VARCHAR)
+.put(PcapTypes.INTEGER, MinorType.INT)
+.put(PcapTypes.LONG, MinorType.BIGINT)
+.put(PcapTypes.TIMESTAMP, MinorType.TIMESTAMP)
+.build();
+  }
+
+  public PcapRecordReader(final String inputPath,
+  final List projectedColumns) {
+try {
+  this.in = new FileInputStream(inputPath);
+  this.decoder = getPacketDecoder();
+  validBytes = in.read(buffer);
+} catch (IOException e) {
+  throw new RuntimeException("File " + inputPath + " not Found");
+}
+setColumns(projectedColumns);
+  }
+
+  @Override
+  public void setup(final OperatorContext context, final OutputMutator 
output) throws ExecutionSetupException {
+this.output = output;
+  }
+
+  @Override
+  public int next() {
+projectedCols = getProjectedColsIfItNull();
+try {
+  return parsePcapFilesAndPutItToTable();
+} catch (IOException io) {
+  throw new RuntimeException("Trouble with reading packets in file!");
--- End diff --

Drill's error message system can use work, but you can help. Rather than 
throw a generic Java exception, please throw a so-called `UserException`:

```
throw 

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-25 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118616240
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.AbstractRecordReader;
+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.dto.ColumnDto;
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+import org.apache.drill.exec.store.pcap.schema.Schema;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.drill.exec.store.pcap.Utils.parseBytesToASCII;
+
+public class PcapRecordReader extends AbstractRecordReader {
+
+  private OutputMutator output;
+
+  private final PacketDecoder decoder;
+  private ImmutableList projectedCols;
+
+  private byte[] buffer = new byte[10];
--- End diff --

Do you want to do this at construct time? If you scan 1000 pcap files in a 
single fragment, Drill will create 1000 record readers at the start of 
execution. Each will allocate a 100K buffer. You'll have 100MB of heap in 
buffers, of which only one will ever be used.

Suggestion: allocate the buffer in setup, clear it in close, so that only 
one buffer is used per fragment.


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-25 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118615528
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatPlugin.java
 ---
@@ -0,0 +1,115 @@
+/*
+ * 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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.store.RecordWriter;
+import org.apache.drill.exec.store.dfs.BasicFormatMatcher;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.FileSelection;
+import org.apache.drill.exec.store.dfs.FileSystemPlugin;
+import org.apache.drill.exec.store.dfs.FormatMatcher;
+import org.apache.drill.exec.store.dfs.FormatSelection;
+import org.apache.drill.exec.store.dfs.MagicString;
+import org.apache.drill.exec.store.dfs.NamedFormatPluginConfig;
+import org.apache.drill.exec.store.dfs.easy.EasyFormatPlugin;
+import org.apache.drill.exec.store.dfs.easy.EasyWriter;
+import org.apache.drill.exec.store.dfs.easy.FileWork;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.regex.Pattern;
+
+public class PcapFormatPlugin extends EasyFormatPlugin {
+
+  private final PcapFormatMatcher matcher;
+
+  public PcapFormatPlugin(String name, DrillbitContext context, 
Configuration fsConf,
+  StoragePluginConfig storagePluginConfig) {
+this(name, context, fsConf, storagePluginConfig, new 
PcapFormatConfig());
+  }
+
+  public PcapFormatPlugin(String name, DrillbitContext context, 
Configuration fsConf, StoragePluginConfig config, PcapFormatConfig 
formatPluginConfig) {
+super(name, context, fsConf, config, formatPluginConfig, true, false, 
true, false, Lists.newArrayList("pcap"), "pcap");
+this.matcher = new PcapFormatMatcher(this);
+  }
+
+  @Override
+  public boolean supportsPushDown() {
+return true;
+  }
+
+  @Override
+  public RecordReader getRecordReader(FragmentContext context, 
DrillFileSystem dfs, FileWork fileWork, List columns, String 
userName) throws ExecutionSetupException {
+String path = dfs.makeQualified(new 
Path(fileWork.getPath())).toUri().getPath();
+return new PcapRecordReader(path, columns);
+  }
+
+  @Override
+  public RecordWriter getRecordWriter(FragmentContext context, EasyWriter 
writer) throws IOException {
+return null;
+  }
+
+  @Override
+  public int getReaderOperatorType() {
+return 0;
+  }
+
+  @Override
+  public int getWriterOperatorType() {
+return 0;
--- End diff --

Other format plugins do the following when a writer is not supported:

```
throw new UnsupportedOperationException("unimplemented");
```


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-25 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118618438
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.AbstractRecordReader;
+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.dto.ColumnDto;
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+import org.apache.drill.exec.store.pcap.schema.Schema;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.drill.exec.store.pcap.Utils.parseBytesToASCII;
+
+public class PcapRecordReader extends AbstractRecordReader {
+
+  private OutputMutator output;
+
+  private final PacketDecoder decoder;
+  private ImmutableList projectedCols;
+
+  private byte[] buffer = new byte[10];
+  private int offset = 0;
+  private InputStream in;
+  private int validBytes;
+
+  private static final Map TYPES;
+
+  private static class ProjectedColumnInfo {
+ValueVector vv;
+ColumnDto pcapColumn;
+  }
+
+  static {
+TYPES = ImmutableMap.builder()
+.put(PcapTypes.STRING, MinorType.VARCHAR)
+.put(PcapTypes.INTEGER, MinorType.INT)
+.put(PcapTypes.LONG, MinorType.BIGINT)
+.put(PcapTypes.TIMESTAMP, MinorType.TIMESTAMP)
+.build();
+  }
+
+  public PcapRecordReader(final String inputPath,
+  final List projectedColumns) {
+try {
+  this.in = new FileInputStream(inputPath);
+  this.decoder = getPacketDecoder();
+  validBytes = in.read(buffer);
+} catch (IOException e) {
+  throw new RuntimeException("File " + inputPath + " not Found");
+}
+setColumns(projectedColumns);
+  }
+
+  @Override
+  public void setup(final OperatorContext context, final OutputMutator 
output) throws ExecutionSetupException {
+this.output = output;
+  }
+
+  @Override
+  public int next() {
+projectedCols = getProjectedColsIfItNull();
+try {
+  return parsePcapFilesAndPutItToTable();
+} catch (IOException io) {
+  throw new RuntimeException("Trouble with reading packets in file!");
+}
+  }
+
+  @Override
+  public void close() throws Exception {
+  }
+
+  private PacketDecoder getPacketDecoder() {
+try {
+  return new PacketDecoder(in);
+ 

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-25 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118616911
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.AbstractRecordReader;
+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.dto.ColumnDto;
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+import org.apache.drill.exec.store.pcap.schema.Schema;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.drill.exec.store.pcap.Utils.parseBytesToASCII;
+
+public class PcapRecordReader extends AbstractRecordReader {
+
+  private OutputMutator output;
+
+  private final PacketDecoder decoder;
+  private ImmutableList projectedCols;
+
+  private byte[] buffer = new byte[10];
+  private int offset = 0;
+  private InputStream in;
+  private int validBytes;
+
+  private static final Map TYPES;
+
+  private static class ProjectedColumnInfo {
+ValueVector vv;
+ColumnDto pcapColumn;
+  }
+
+  static {
+TYPES = ImmutableMap.builder()
+.put(PcapTypes.STRING, MinorType.VARCHAR)
+.put(PcapTypes.INTEGER, MinorType.INT)
+.put(PcapTypes.LONG, MinorType.BIGINT)
+.put(PcapTypes.TIMESTAMP, MinorType.TIMESTAMP)
+.build();
+  }
+
+  public PcapRecordReader(final String inputPath,
+  final List projectedColumns) {
+try {
+  this.in = new FileInputStream(inputPath);
+  this.decoder = getPacketDecoder();
+  validBytes = in.read(buffer);
+} catch (IOException e) {
+  throw new RuntimeException("File " + inputPath + " not Found");
+}
+setColumns(projectedColumns);
+  }
+
+  @Override
+  public void setup(final OperatorContext context, final OutputMutator 
output) throws ExecutionSetupException {
+this.output = output;
+  }
+
+  @Override
+  public int next() {
+projectedCols = getProjectedColsIfItNull();
+try {
+  return parsePcapFilesAndPutItToTable();
+} catch (IOException io) {
+  throw new RuntimeException("Trouble with reading packets in file!");
+}
+  }
+
+  @Override
+  public void close() throws Exception {
+  }
+
+  private PacketDecoder getPacketDecoder() {
+try {
+  return new PacketDecoder(in);
+ 

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-25 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118619851
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/Packet.java
 ---
@@ -0,0 +1,371 @@
+/*
+ * 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.decoder;
+
+import com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import static org.apache.drill.exec.store.pcap.Utils.convertInt;
+import static org.apache.drill.exec.store.pcap.Utils.convertShort;
+import static org.apache.drill.exec.store.pcap.Utils.getByte;
+import static org.apache.drill.exec.store.pcap.Utils.getIntFileOrder;
+import static org.apache.drill.exec.store.pcap.Utils.getShort;
+
+public class Packet {
--- End diff --

Would it have been possible to use one of the existing pcap Java libraries 
here? Four are listed [here](https://en.wikipedia.org/wiki/Pcap).


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-25 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118615907
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatPlugin.java
 ---
@@ -0,0 +1,115 @@
+/*
+ * 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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.store.RecordWriter;
+import org.apache.drill.exec.store.dfs.BasicFormatMatcher;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.FileSelection;
+import org.apache.drill.exec.store.dfs.FileSystemPlugin;
+import org.apache.drill.exec.store.dfs.FormatMatcher;
+import org.apache.drill.exec.store.dfs.FormatSelection;
+import org.apache.drill.exec.store.dfs.MagicString;
+import org.apache.drill.exec.store.dfs.NamedFormatPluginConfig;
+import org.apache.drill.exec.store.dfs.easy.EasyFormatPlugin;
+import org.apache.drill.exec.store.dfs.easy.EasyWriter;
+import org.apache.drill.exec.store.dfs.easy.FileWork;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.regex.Pattern;
+
+public class PcapFormatPlugin extends EasyFormatPlugin {
+
+  private final PcapFormatMatcher matcher;
+
+  public PcapFormatPlugin(String name, DrillbitContext context, 
Configuration fsConf,
+  StoragePluginConfig storagePluginConfig) {
+this(name, context, fsConf, storagePluginConfig, new 
PcapFormatConfig());
+  }
+
+  public PcapFormatPlugin(String name, DrillbitContext context, 
Configuration fsConf, StoragePluginConfig config, PcapFormatConfig 
formatPluginConfig) {
+super(name, context, fsConf, config, formatPluginConfig, true, false, 
true, false, Lists.newArrayList("pcap"), "pcap");
+this.matcher = new PcapFormatMatcher(this);
+  }
+
+  @Override
+  public boolean supportsPushDown() {
+return true;
+  }
+
+  @Override
+  public RecordReader getRecordReader(FragmentContext context, 
DrillFileSystem dfs, FileWork fileWork, List columns, String 
userName) throws ExecutionSetupException {
+String path = dfs.makeQualified(new 
Path(fileWork.getPath())).toUri().getPath();
+return new PcapRecordReader(path, columns);
+  }
+
+  @Override
+  public RecordWriter getRecordWriter(FragmentContext context, EasyWriter 
writer) throws IOException {
+return null;
+  }
+
+  @Override
+  public int getReaderOperatorType() {
+return 0;
--- End diff --

Seems akward, but it seems that other format plugins add a type to a 
protobuf, then return that here:

```
return CoreOperatorType.JSON_SUB_SCAN_VALUE;
```

And `UserBitShared.proto`:

```
  JSON_SUB_SCAN = 29;
```

The next available number is 37.

This seems rather brittle. Seems we should have a more general solution. 
But, until we do, I'd guess you'll need to add the enum value.

As an alternative, `SequenceFileForamtPlugin` just makes up a number:

```
  public int getReaderOperatorType() {
return 4001;
  }
```


---
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 

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-24 Thread parthchandra
Github user parthchandra commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118406754
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/Murmur128.java
 ---
@@ -0,0 +1,161 @@
+/*
--- End diff --

I was thinking of simply moving the code to the existing implementation. 


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-24 Thread parthchandra
Github user parthchandra commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118406626
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/dto/ColumnDto.java
 ---
@@ -0,0 +1,63 @@
+/*
+ * 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.dto;
+
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+
+import java.util.Objects;
+
+public class ColumnDto {
+
+  private final String columnName;
+  private final PcapTypes columnType;
+
+  public ColumnDto(String columnName, PcapTypes columnType) {
+this.columnName = columnName;
+this.columnType = columnType;
+  }
+
+  public String getColumnName() {
+return columnName;
+  }
+
+  public PcapTypes getColumnType() {
+return columnType;
+  }
+
+  public boolean isNullable() {
+return true;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+if (this == o) {
+  return true;
+}
+if (o == null || getClass() != o.getClass()) {
+  return false;
+}
+ColumnDto columnDto = (ColumnDto) o;
+return Objects.equals(columnName, columnDto.columnName) &&
--- End diff --

OK, my mistake. The code is `Objects.equals` and not `Object.equals` so is 
in fact correct.


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-24 Thread parthchandra
Github user parthchandra commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118406369
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/Packet.java
 ---
@@ -0,0 +1,371 @@
+/*
+ * 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.decoder;
+
+import com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import static org.apache.drill.exec.store.pcap.Utils.convertInt;
+import static org.apache.drill.exec.store.pcap.Utils.convertShort;
+import static org.apache.drill.exec.store.pcap.Utils.getByte;
+import static org.apache.drill.exec.store.pcap.Utils.getIntFileOrder;
+import static org.apache.drill.exec.store.pcap.Utils.getShort;
+
+public class Packet {
+  // pcap header
+  //typedef struct pcaprec_hdr_s {
+  //guint32 ts_sec; // timestamp seconds
+  //guint32 ts_usec;// timestamp microseconds */
+  //guint32 incl_len;   // number of octets of packet 
saved in file */
+  //guint32 orig_len;   // actual length of packet */
+  //} pcaprec_hdr_t;
+  private long timestamp;
+  private int originalLength;
+
+  private byte[] raw;
+
+  private int etherOffset;
+  private int ipOffset;
+
+  private int packetLength;
+  private int etherProtocol;
+  private int protocol;
+
+  private boolean isRoutingV6;
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean readPcap(final InputStream in, final boolean byteOrder, 
final int maxLength) throws IOException {
+byte[] pcapHeader = new byte[PacketConstants.PCAP_HEADER_SIZE];
+int n = in.read(pcapHeader);
+if (n < pcapHeader.length) {
+  return false;
+}
+decodePcapHeader(pcapHeader, byteOrder, maxLength, 0);
+
+raw = new byte[originalLength];
+n = in.read(raw);
+if (n < 0) {
+  return false;
+}
+etherOffset = 0;
+
+decodeEtherPacket();
+return true;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public int decodePcap(final byte[] buffer, final int offset, final 
boolean byteOrder, final int maxLength) {
+raw = buffer;
+etherOffset = offset + PacketConstants.PCAP_HEADER_SIZE;
+decodePcapHeader(raw, byteOrder, maxLength, offset);
+decodeEtherPacket();
+return offset + PacketConstants.PCAP_HEADER_SIZE + originalLength;
+  }
+
+  public String getPacketType() {
+if (isTcpPacket()) {
+  return "TCP";
+} else if (isUdpPacket()) {
+  return "UDP";
+} else if (isArpPacket()) {
+  return "ARP";
+} else if (isIcmpPacket()) {
+  return "ICMP";
+} else {
+  return "unknown";
+}
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isIpV4Packet() {
+return etherProtocol == PacketConstants.IPv4_TYPE;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isIpV6Packet() {
+return etherProtocol == PacketConstants.IPv6_TYPE;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isPPPoV6Packet() {
+return etherProtocol == PacketConstants.PPPoV6_TYPE;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isTcpPacket() {
+return protocol == PacketConstants.TCP_PROTOCOL;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isUdpPacket() {
+return protocol == PacketConstants.UDP_PROTOCOL;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isArpPacket() {
+return protocol == PacketConstants.ARP_PROTOCOL;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isIcmpPacket() {
+return protocol == 

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-24 Thread tdunning
Github user tdunning commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118399899
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.AbstractRecordReader;
+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.dto.ColumnDto;
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+import org.apache.drill.exec.store.pcap.schema.Schema;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.drill.exec.store.pcap.Utils.parseBytesToASCII;
+
+public class PcapRecordReader extends AbstractRecordReader {
+
+  private OutputMutator output;
+
+  private final PacketDecoder decoder;
+  private ImmutableList projectedCols;
+
+  private byte[] buffer = new byte[10];
+  private int offset = 0;
+  private InputStream in;
+  private int validBytes;
+
+  private static final Map TYPES;
+
+  private static class ProjectedColumnInfo {
+ValueVector vv;
+ColumnDto pcapColumn;
+  }
+
+  static {
+TYPES = ImmutableMap.builder()
+.put(PcapTypes.STRING, MinorType.VARCHAR)
+.put(PcapTypes.INTEGER, MinorType.INT)
+.put(PcapTypes.LONG, MinorType.BIGINT)
+.put(PcapTypes.TIMESTAMP, MinorType.TIMESTAMP)
+.build();
+  }
+
+  public PcapRecordReader(final String inputPath,
+  final List projectedColumns) {
+try {
+  this.in = new FileInputStream(inputPath);
+  this.decoder = getPacketDecoder();
+  validBytes = in.read(buffer);
+} catch (IOException e) {
+  throw new RuntimeException("File " + inputPath + " not Found");
+}
+setColumns(projectedColumns);
+  }
+
+  @Override
+  public void setup(final OperatorContext context, final OutputMutator 
output) throws ExecutionSetupException {
+this.output = output;
+  }
+
+  @Override
+  public int next() {
+projectedCols = getProjectedColsIfItNull();
+try {
+  return parsePcapFilesAndPutItToTable();
+} catch (IOException io) {
+  throw new RuntimeException("Trouble with reading packets in file!");
+}
+  }
+
+  @Override
+  public void close() throws Exception {
+  }
+
+  private PacketDecoder getPacketDecoder() {
+try {
+  return new PacketDecoder(in);
+

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-24 Thread tdunning
Github user tdunning commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118399867
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/Packet.java
 ---
@@ -0,0 +1,371 @@
+/*
+ * 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.decoder;
+
+import com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import static org.apache.drill.exec.store.pcap.Utils.convertInt;
+import static org.apache.drill.exec.store.pcap.Utils.convertShort;
+import static org.apache.drill.exec.store.pcap.Utils.getByte;
+import static org.apache.drill.exec.store.pcap.Utils.getIntFileOrder;
+import static org.apache.drill.exec.store.pcap.Utils.getShort;
+
+public class Packet {
+  // pcap header
+  //typedef struct pcaprec_hdr_s {
+  //guint32 ts_sec; // timestamp seconds
+  //guint32 ts_usec;// timestamp microseconds */
+  //guint32 incl_len;   // number of octets of packet 
saved in file */
+  //guint32 orig_len;   // actual length of packet */
+  //} pcaprec_hdr_t;
+  private long timestamp;
+  private int originalLength;
+
+  private byte[] raw;
+
+  private int etherOffset;
+  private int ipOffset;
+
+  private int packetLength;
+  private int etherProtocol;
+  private int protocol;
+
+  private boolean isRoutingV6;
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean readPcap(final InputStream in, final boolean byteOrder, 
final int maxLength) throws IOException {
+byte[] pcapHeader = new byte[PacketConstants.PCAP_HEADER_SIZE];
+int n = in.read(pcapHeader);
+if (n < pcapHeader.length) {
+  return false;
+}
+decodePcapHeader(pcapHeader, byteOrder, maxLength, 0);
+
+raw = new byte[originalLength];
+n = in.read(raw);
+if (n < 0) {
+  return false;
+}
+etherOffset = 0;
+
+decodeEtherPacket();
+return true;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public int decodePcap(final byte[] buffer, final int offset, final 
boolean byteOrder, final int maxLength) {
+raw = buffer;
+etherOffset = offset + PacketConstants.PCAP_HEADER_SIZE;
+decodePcapHeader(raw, byteOrder, maxLength, offset);
+decodeEtherPacket();
+return offset + PacketConstants.PCAP_HEADER_SIZE + originalLength;
+  }
+
+  public String getPacketType() {
+if (isTcpPacket()) {
+  return "TCP";
+} else if (isUdpPacket()) {
+  return "UDP";
+} else if (isArpPacket()) {
+  return "ARP";
+} else if (isIcmpPacket()) {
+  return "ICMP";
+} else {
+  return "unknown";
+}
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isIpV4Packet() {
+return etherProtocol == PacketConstants.IPv4_TYPE;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isIpV6Packet() {
+return etherProtocol == PacketConstants.IPv6_TYPE;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isPPPoV6Packet() {
+return etherProtocol == PacketConstants.PPPoV6_TYPE;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isTcpPacket() {
+return protocol == PacketConstants.TCP_PROTOCOL;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isUdpPacket() {
+return protocol == PacketConstants.UDP_PROTOCOL;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isArpPacket() {
+return protocol == PacketConstants.ARP_PROTOCOL;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isIcmpPacket() {
+return protocol == 

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-24 Thread tdunning
Github user tdunning commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118399621
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/Packet.java
 ---
@@ -0,0 +1,371 @@
+/*
+ * 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.decoder;
+
+import com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import static org.apache.drill.exec.store.pcap.Utils.convertInt;
+import static org.apache.drill.exec.store.pcap.Utils.convertShort;
+import static org.apache.drill.exec.store.pcap.Utils.getByte;
+import static org.apache.drill.exec.store.pcap.Utils.getIntFileOrder;
+import static org.apache.drill.exec.store.pcap.Utils.getShort;
+
+public class Packet {
+  // pcap header
+  //typedef struct pcaprec_hdr_s {
+  //guint32 ts_sec; // timestamp seconds
+  //guint32 ts_usec;// timestamp microseconds */
+  //guint32 incl_len;   // number of octets of packet 
saved in file */
+  //guint32 orig_len;   // actual length of packet */
+  //} pcaprec_hdr_t;
+  private long timestamp;
+  private int originalLength;
+
+  private byte[] raw;
+
+  private int etherOffset;
+  private int ipOffset;
+
+  private int packetLength;
+  private int etherProtocol;
+  private int protocol;
+
+  private boolean isRoutingV6;
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean readPcap(final InputStream in, final boolean byteOrder, 
final int maxLength) throws IOException {
+byte[] pcapHeader = new byte[PacketConstants.PCAP_HEADER_SIZE];
+int n = in.read(pcapHeader);
+if (n < pcapHeader.length) {
+  return false;
+}
+decodePcapHeader(pcapHeader, byteOrder, maxLength, 0);
+
+raw = new byte[originalLength];
+n = in.read(raw);
+if (n < 0) {
+  return false;
+}
+etherOffset = 0;
+
+decodeEtherPacket();
+return true;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public int decodePcap(final byte[] buffer, final int offset, final 
boolean byteOrder, final int maxLength) {
+raw = buffer;
+etherOffset = offset + PacketConstants.PCAP_HEADER_SIZE;
+decodePcapHeader(raw, byteOrder, maxLength, offset);
+decodeEtherPacket();
+return offset + PacketConstants.PCAP_HEADER_SIZE + originalLength;
+  }
+
+  public String getPacketType() {
+if (isTcpPacket()) {
+  return "TCP";
+} else if (isUdpPacket()) {
+  return "UDP";
+} else if (isArpPacket()) {
+  return "ARP";
+} else if (isIcmpPacket()) {
+  return "ICMP";
+} else {
+  return "unknown";
+}
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isIpV4Packet() {
+return etherProtocol == PacketConstants.IPv4_TYPE;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isIpV6Packet() {
+return etherProtocol == PacketConstants.IPv6_TYPE;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isPPPoV6Packet() {
+return etherProtocol == PacketConstants.PPPoV6_TYPE;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isTcpPacket() {
+return protocol == PacketConstants.TCP_PROTOCOL;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isUdpPacket() {
+return protocol == PacketConstants.UDP_PROTOCOL;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isArpPacket() {
+return protocol == PacketConstants.ARP_PROTOCOL;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isIcmpPacket() {
+return protocol == 

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-24 Thread tdunning
Github user tdunning commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118399563
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapDecoder.java
 ---
@@ -0,0 +1,230 @@
+/*
+ * 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 com.google.common.io.Resources;
+import org.apache.drill.BaseTestQuery;
+import org.apache.drill.exec.store.pcap.decoder.Packet;
+import org.apache.drill.exec.store.pcap.decoder.PacketDecoder;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.BufferedInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestPcapDecoder extends BaseTestQuery {
+  private static File bigFile;
+
+  /**
+   * Creates an ephemeral file of about a GB in size
+   *
+   * @throws IOException If input file can't be read or output can't be 
written.
+   */
+  @BeforeClass
+  public static void buildBigTcpFile() throws IOException {
+bigFile = File.createTempFile("tcp", ".pcap");
+bigFile.deleteOnExit();
+boolean first = true;
+System.out.printf("Building large test file\n");
+try (DataOutputStream out = new DataOutputStream(new 
FileOutputStream(bigFile))) {
+  for (int i = 0; i < 1000e6 / (29208 - 24) + 1; i++) {
+// might be faster to keep this open and rewind each time, but
+// that is hard to do with a resource, especially if it comes
+// from the class path instead of files.
+try (InputStream in = 
Resources.getResource("store/pcap/tcp-2.pcap").openStream()) {
+  ConcatPcap.copy(first, in, out);
+}
+first = false;
+  }
+  System.out.printf("Created file is %.1f MB\n", bigFile.length() / 
1e6);
--- End diff --

I changed those methods to be called from a public static void main(). That 
allows them to be used to get information about speeds, but doesn't include 
their output in the test.

I think that addresses this comment.


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-24 Thread tdunning
Github user tdunning commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118397733
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapDecoder.java
 ---
@@ -0,0 +1,230 @@
+/*
+ * 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 com.google.common.io.Resources;
+import org.apache.drill.BaseTestQuery;
+import org.apache.drill.exec.store.pcap.decoder.Packet;
+import org.apache.drill.exec.store.pcap.decoder.PacketDecoder;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.BufferedInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestPcapDecoder extends BaseTestQuery {
+  private static File bigFile;
+
+  /**
+   * Creates an ephemeral file of about a GB in size
+   *
+   * @throws IOException If input file can't be read or output can't be 
written.
+   */
+  @BeforeClass
+  public static void buildBigTcpFile() throws IOException {
+bigFile = File.createTempFile("tcp", ".pcap");
+bigFile.deleteOnExit();
+boolean first = true;
+System.out.printf("Building large test file\n");
+try (DataOutputStream out = new DataOutputStream(new 
FileOutputStream(bigFile))) {
+  for (int i = 0; i < 1000e6 / (29208 - 24) + 1; i++) {
+// might be faster to keep this open and rewind each time, but
+// that is hard to do with a resource, especially if it comes
+// from the class path instead of files.
+try (InputStream in = 
Resources.getResource("store/pcap/tcp-2.pcap").openStream()) {
+  ConcatPcap.copy(first, in, out);
+}
+first = false;
+  }
+  System.out.printf("Created file is %.1f MB\n", bigFile.length() / 
1e6);
--- End diff --

Sure.  This good and easy to delete.


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-24 Thread tdunning
Github user tdunning commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118397652
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.AbstractRecordReader;
+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.dto.ColumnDto;
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+import org.apache.drill.exec.store.pcap.schema.Schema;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.drill.exec.store.pcap.Utils.parseBytesToASCII;
+
+public class PcapRecordReader extends AbstractRecordReader {
+
+  private OutputMutator output;
+
+  private final PacketDecoder decoder;
+  private ImmutableList projectedCols;
+
+  private byte[] buffer = new byte[10];
+  private int offset = 0;
+  private InputStream in;
+  private int validBytes;
+
+  private static final Map TYPES;
+
+  private static class ProjectedColumnInfo {
+ValueVector vv;
+ColumnDto pcapColumn;
+  }
+
+  static {
+TYPES = ImmutableMap.builder()
+.put(PcapTypes.STRING, MinorType.VARCHAR)
+.put(PcapTypes.INTEGER, MinorType.INT)
+.put(PcapTypes.LONG, MinorType.BIGINT)
+.put(PcapTypes.TIMESTAMP, MinorType.TIMESTAMP)
+.build();
+  }
+
+  public PcapRecordReader(final String inputPath,
+  final List projectedColumns) {
+try {
+  this.in = new FileInputStream(inputPath);
+  this.decoder = getPacketDecoder();
+  validBytes = in.read(buffer);
+} catch (IOException e) {
+  throw new RuntimeException("File " + inputPath + " not Found");
+}
+setColumns(projectedColumns);
+  }
+
+  @Override
+  public void setup(final OperatorContext context, final OutputMutator 
output) throws ExecutionSetupException {
+this.output = output;
+  }
+
+  @Override
+  public int next() {
+projectedCols = getProjectedColsIfItNull();
+try {
+  return parsePcapFilesAndPutItToTable();
+} catch (IOException io) {
+  throw new RuntimeException("Trouble with reading packets in file!");
+}
+  }
+
+  @Override
+  public void close() throws Exception {
+  }
+
+  private PacketDecoder getPacketDecoder() {
+try {
+  return new PacketDecoder(in);
+

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-24 Thread tdunning
Github user tdunning commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118397540
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/Murmur128.java
 ---
@@ -0,0 +1,161 @@
+/*
--- End diff --

I tried to merge them, but the assumptions of working on unsafe code were 
very difficult to remove. I also had a difficult time figuring out a useful API.

Happy to look at specific suggestions, but the way that the existing 
implementation always iterates through unsafe pointers meant that there is 
likely to be nearly no shared code. Merging by simply moving my methods to the 
existing implementation is certainly doable.



---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-23 Thread parthchandra
Github user parthchandra commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118108814
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapRecordReader.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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.AbstractRecordReader;
+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.dto.ColumnDto;
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+import org.apache.drill.exec.store.pcap.schema.Schema;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.drill.exec.store.pcap.Utils.parseBytesToASCII;
+
+public class PcapRecordReader extends AbstractRecordReader {
+
+  private OutputMutator output;
+
+  private final PacketDecoder decoder;
+  private ImmutableList projectedCols;
+
+  private byte[] buffer = new byte[10];
+  private int offset = 0;
+  private InputStream in;
+  private int validBytes;
+
+  private static final Map TYPES;
+
+  private static class ProjectedColumnInfo {
+ValueVector vv;
+ColumnDto pcapColumn;
+  }
+
+  static {
+TYPES = ImmutableMap.builder()
+.put(PcapTypes.STRING, MinorType.VARCHAR)
+.put(PcapTypes.INTEGER, MinorType.INT)
+.put(PcapTypes.LONG, MinorType.BIGINT)
+.put(PcapTypes.TIMESTAMP, MinorType.TIMESTAMP)
+.build();
+  }
+
+  public PcapRecordReader(final String inputPath,
+  final List projectedColumns) {
+try {
+  this.in = new FileInputStream(inputPath);
+  this.decoder = getPacketDecoder();
+  validBytes = in.read(buffer);
+} catch (IOException e) {
+  throw new RuntimeException("File " + inputPath + " not Found");
+}
+setColumns(projectedColumns);
+  }
+
+  @Override
+  public void setup(final OperatorContext context, final OutputMutator 
output) throws ExecutionSetupException {
+this.output = output;
+  }
+
+  @Override
+  public int next() {
+projectedCols = getProjectedColsIfItNull();
+try {
+  return parsePcapFilesAndPutItToTable();
+} catch (IOException io) {
+  throw new RuntimeException("Trouble with reading packets in file!");
+}
+  }
+
+  @Override
+  public void close() throws Exception {
+  }
+
+  private PacketDecoder getPacketDecoder() {
+try {
+  return new PacketDecoder(in);

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-23 Thread parthchandra
Github user parthchandra commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118097440
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/dto/ColumnDto.java
 ---
@@ -0,0 +1,63 @@
+/*
+ * 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.dto;
+
+import org.apache.drill.exec.store.pcap.schema.PcapTypes;
+
+import java.util.Objects;
+
+public class ColumnDto {
+
+  private final String columnName;
+  private final PcapTypes columnType;
+
+  public ColumnDto(String columnName, PcapTypes columnType) {
+this.columnName = columnName;
+this.columnType = columnType;
+  }
+
+  public String getColumnName() {
+return columnName;
+  }
+
+  public PcapTypes getColumnType() {
+return columnType;
+  }
+
+  public boolean isNullable() {
+return true;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+if (this == o) {
+  return true;
+}
+if (o == null || getClass() != o.getClass()) {
+  return false;
+}
+ColumnDto columnDto = (ColumnDto) o;
+return Objects.equals(columnName, columnDto.columnName) &&
--- End diff --

Are you sure `Object.equals` is what you want here, and not `String.equals` 
?


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-23 Thread parthchandra
Github user parthchandra commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118098082
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestPcapDecoder.java
 ---
@@ -0,0 +1,230 @@
+/*
+ * 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 com.google.common.io.Resources;
+import org.apache.drill.BaseTestQuery;
+import org.apache.drill.exec.store.pcap.decoder.Packet;
+import org.apache.drill.exec.store.pcap.decoder.PacketDecoder;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.BufferedInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestPcapDecoder extends BaseTestQuery {
+  private static File bigFile;
+
+  /**
+   * Creates an ephemeral file of about a GB in size
+   *
+   * @throws IOException If input file can't be read or output can't be 
written.
+   */
+  @BeforeClass
+  public static void buildBigTcpFile() throws IOException {
+bigFile = File.createTempFile("tcp", ".pcap");
+bigFile.deleteOnExit();
+boolean first = true;
+System.out.printf("Building large test file\n");
+try (DataOutputStream out = new DataOutputStream(new 
FileOutputStream(bigFile))) {
+  for (int i = 0; i < 1000e6 / (29208 - 24) + 1; i++) {
+// might be faster to keep this open and rewind each time, but
+// that is hard to do with a resource, especially if it comes
+// from the class path instead of files.
+try (InputStream in = 
Resources.getResource("store/pcap/tcp-2.pcap").openStream()) {
+  ConcatPcap.copy(first, in, out);
+}
+first = false;
+  }
+  System.out.printf("Created file is %.1f MB\n", bigFile.length() / 
1e6);
--- End diff --

Can we not use System.out in a test? We're trying to get the output from a 
full build to be smaller. 


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-23 Thread parthchandra
Github user parthchandra commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118107316
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/Packet.java
 ---
@@ -0,0 +1,371 @@
+/*
+ * 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.decoder;
+
+import com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import static org.apache.drill.exec.store.pcap.Utils.convertInt;
+import static org.apache.drill.exec.store.pcap.Utils.convertShort;
+import static org.apache.drill.exec.store.pcap.Utils.getByte;
+import static org.apache.drill.exec.store.pcap.Utils.getIntFileOrder;
+import static org.apache.drill.exec.store.pcap.Utils.getShort;
+
+public class Packet {
+  // pcap header
+  //typedef struct pcaprec_hdr_s {
+  //guint32 ts_sec; // timestamp seconds
+  //guint32 ts_usec;// timestamp microseconds */
+  //guint32 incl_len;   // number of octets of packet 
saved in file */
+  //guint32 orig_len;   // actual length of packet */
+  //} pcaprec_hdr_t;
+  private long timestamp;
+  private int originalLength;
+
+  private byte[] raw;
+
+  private int etherOffset;
+  private int ipOffset;
+
+  private int packetLength;
+  private int etherProtocol;
+  private int protocol;
+
+  private boolean isRoutingV6;
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean readPcap(final InputStream in, final boolean byteOrder, 
final int maxLength) throws IOException {
+byte[] pcapHeader = new byte[PacketConstants.PCAP_HEADER_SIZE];
+int n = in.read(pcapHeader);
+if (n < pcapHeader.length) {
+  return false;
+}
+decodePcapHeader(pcapHeader, byteOrder, maxLength, 0);
+
+raw = new byte[originalLength];
+n = in.read(raw);
+if (n < 0) {
+  return false;
+}
+etherOffset = 0;
+
+decodeEtherPacket();
+return true;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public int decodePcap(final byte[] buffer, final int offset, final 
boolean byteOrder, final int maxLength) {
+raw = buffer;
+etherOffset = offset + PacketConstants.PCAP_HEADER_SIZE;
+decodePcapHeader(raw, byteOrder, maxLength, offset);
+decodeEtherPacket();
+return offset + PacketConstants.PCAP_HEADER_SIZE + originalLength;
+  }
+
+  public String getPacketType() {
+if (isTcpPacket()) {
+  return "TCP";
+} else if (isUdpPacket()) {
+  return "UDP";
+} else if (isArpPacket()) {
+  return "ARP";
+} else if (isIcmpPacket()) {
+  return "ICMP";
+} else {
+  return "unknown";
+}
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isIpV4Packet() {
+return etherProtocol == PacketConstants.IPv4_TYPE;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isIpV6Packet() {
+return etherProtocol == PacketConstants.IPv6_TYPE;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isPPPoV6Packet() {
+return etherProtocol == PacketConstants.PPPoV6_TYPE;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isTcpPacket() {
+return protocol == PacketConstants.TCP_PROTOCOL;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isUdpPacket() {
+return protocol == PacketConstants.UDP_PROTOCOL;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isArpPacket() {
+return protocol == PacketConstants.ARP_PROTOCOL;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isIcmpPacket() {
+return protocol == 

[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-23 Thread parthchandra
Github user parthchandra commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118109637
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/Murmur128.java
 ---
@@ -0,0 +1,161 @@
+/*
--- End diff --

We already have a Murmur Hash implementation for use with Direct buffers. 
(see `org.apache.drill.exec.expr.fn.impl.MurmurHash3` ). Can we merge these two 
and have only one copy?


---
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.
---


[GitHub] drill pull request #831: DRILL-5432: Added pcap-format support

2017-05-23 Thread parthchandra
Github user parthchandra commented on a diff in the pull request:

https://github.com/apache/drill/pull/831#discussion_r118102624
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/decoder/Packet.java
 ---
@@ -0,0 +1,371 @@
+/*
+ * 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.decoder;
+
+import com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import static org.apache.drill.exec.store.pcap.Utils.convertInt;
+import static org.apache.drill.exec.store.pcap.Utils.convertShort;
+import static org.apache.drill.exec.store.pcap.Utils.getByte;
+import static org.apache.drill.exec.store.pcap.Utils.getIntFileOrder;
+import static org.apache.drill.exec.store.pcap.Utils.getShort;
+
+public class Packet {
+  // pcap header
+  //typedef struct pcaprec_hdr_s {
+  //guint32 ts_sec; // timestamp seconds
+  //guint32 ts_usec;// timestamp microseconds */
+  //guint32 incl_len;   // number of octets of packet 
saved in file */
+  //guint32 orig_len;   // actual length of packet */
+  //} pcaprec_hdr_t;
+  private long timestamp;
+  private int originalLength;
+
+  private byte[] raw;
+
+  private int etherOffset;
+  private int ipOffset;
+
+  private int packetLength;
+  private int etherProtocol;
+  private int protocol;
+
+  private boolean isRoutingV6;
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean readPcap(final InputStream in, final boolean byteOrder, 
final int maxLength) throws IOException {
+byte[] pcapHeader = new byte[PacketConstants.PCAP_HEADER_SIZE];
+int n = in.read(pcapHeader);
+if (n < pcapHeader.length) {
+  return false;
+}
+decodePcapHeader(pcapHeader, byteOrder, maxLength, 0);
+
+raw = new byte[originalLength];
+n = in.read(raw);
+if (n < 0) {
+  return false;
+}
+etherOffset = 0;
+
+decodeEtherPacket();
+return true;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public int decodePcap(final byte[] buffer, final int offset, final 
boolean byteOrder, final int maxLength) {
+raw = buffer;
+etherOffset = offset + PacketConstants.PCAP_HEADER_SIZE;
+decodePcapHeader(raw, byteOrder, maxLength, offset);
+decodeEtherPacket();
+return offset + PacketConstants.PCAP_HEADER_SIZE + originalLength;
+  }
+
+  public String getPacketType() {
+if (isTcpPacket()) {
+  return "TCP";
+} else if (isUdpPacket()) {
+  return "UDP";
+} else if (isArpPacket()) {
+  return "ARP";
+} else if (isIcmpPacket()) {
+  return "ICMP";
+} else {
+  return "unknown";
+}
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isIpV4Packet() {
+return etherProtocol == PacketConstants.IPv4_TYPE;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isIpV6Packet() {
+return etherProtocol == PacketConstants.IPv6_TYPE;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isPPPoV6Packet() {
+return etherProtocol == PacketConstants.PPPoV6_TYPE;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isTcpPacket() {
+return protocol == PacketConstants.TCP_PROTOCOL;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isUdpPacket() {
+return protocol == PacketConstants.UDP_PROTOCOL;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isArpPacket() {
+return protocol == PacketConstants.ARP_PROTOCOL;
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isIcmpPacket() {
+return protocol ==