drill-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (DRILL-5432) Want a memory format for PCAP files
Date Fri, 26 May 2017 01:45:04 GMT

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

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

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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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<PcapFormatConfig> {
    +
    +  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<SchemaPath> 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;
      }
    ```


> Want a memory format for PCAP files
> -----------------------------------
>
>                 Key: DRILL-5432
>                 URL: https://issues.apache.org/jira/browse/DRILL-5432
>             Project: Apache Drill
>          Issue Type: New Feature
>            Reporter: Ted Dunning
>
> PCAP files [1] are the de facto standard for storing network capture data. In security
and protocol applications, it is very common to want to extract particular packets from a
capture for further analysis.
> At a first level, it is desirable to query and filter by source and destination IP and
port or by protocol. Beyond that, however, it would be very useful to be able to group packets
by TCP session and eventually to look at packet contents. For now, however, the most critical
requirement is that we should be able to scan captures at very high speed.
> I previously wrote a (kind of working) proof of concept for a PCAP decoder that did lazy
deserialization and could traverse hundreds of MB of PCAP data per second per core. This compares
to roughly 2-3 MB/s for widely available Apache-compatible open source PCAP decoders.
> This JIRA covers the integration and extension of that proof of concept as a Drill file
format.
> Initial work is available at https://github.com/mapr-demos/drill-pcap-format
> [1] https://en.wikipedia.org/wiki/Pcap



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Mime
View raw message