You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@plc4x.apache.org by ld...@apache.org on 2020/05/11 10:59:01 UTC

[plc4x] branch feature/profinet updated (b69688a -> f587479)

This is an automated email from the ASF dual-hosted git repository.

ldywicki pushed a change to branch feature/profinet
in repository https://gitbox.apache.org/repos/asf/plc4x.git.


    from b69688a  Sketch of Profinet DCP mspec.
     new 30f52a1  First Profinet Discovery and Configuration Protocol implementation.
     new f587479  Implementation of writing part for pcap4j transport.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../rawsocket/RawSocketChannelFactory.java         |   9 +-
 .../utils/rawsockets/netty/RawSocketChannel.java   |  28 +++--
 .../protocols/profinet/profinet.dcp.mspec          |   7 ++
 .../java/profinet/dcp/ProfinetDCPPlcDriver.java    | 103 ++++++++++++++++++
 .../dcp/configuration/ProfinetConfiguration.java   |  53 +++++++++
 .../profinet/dcp/field/ProfinetFieldHandler.java}  |  17 ++-
 .../dcp/protocol/ProfinetDCPProtocolLogic.java     | 120 +++++++++++++++++++++
 .../services/org.apache.plc4x.java.api.PlcDriver   |   2 +-
 8 files changed, 315 insertions(+), 24 deletions(-)
 create mode 100644 sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/ProfinetDCPPlcDriver.java
 create mode 100644 sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/configuration/ProfinetConfiguration.java
 copy sandbox/{test-java-df1-driver/src/main/java/org/apache/plc4x/java/df1/field/Df1FieldHandler.java => test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/field/ProfinetFieldHandler.java} (74%)
 create mode 100644 sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/protocol/ProfinetDCPProtocolLogic.java
 copy {plc4j/drivers/ads => sandbox/test-java-profinet-driver}/src/main/resources/META-INF/services/org.apache.plc4x.java.api.PlcDriver (93%)


[plc4x] 01/02: First Profinet Discovery and Configuration Protocol implementation.

Posted by ld...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ldywicki pushed a commit to branch feature/profinet
in repository https://gitbox.apache.org/repos/asf/plc4x.git

commit 30f52a149203bfb1aeb389550ffa4ad243c8051c
Author: Łukasz Dywicki <lu...@code-house.org>
AuthorDate: Mon May 11 12:46:49 2020 +0200

    First Profinet Discovery and Configuration Protocol implementation.
    
    It is quite basic in ints form, allows to send broadcast over ethernet interface. Responses are not processed, they are just printed to logs.
---
 .../protocols/profinet/profinet.dcp.mspec          |   7 ++
 .../java/profinet/dcp/ProfinetDCPPlcDriver.java    | 103 ++++++++++++++++++
 .../dcp/configuration/ProfinetConfiguration.java   |  53 +++++++++
 .../profinet/dcp/field/ProfinetFieldHandler.java   |  36 +++++++
 .../dcp/protocol/ProfinetDCPProtocolLogic.java     | 120 +++++++++++++++++++++
 .../services/org.apache.plc4x.java.api.PlcDriver   |  19 ++++
 6 files changed, 338 insertions(+)

diff --git a/protocols/profinet/src/main/resources/protocols/profinet/profinet.dcp.mspec b/protocols/profinet/src/main/resources/protocols/profinet/profinet.dcp.mspec
index 298243a..fd0be03 100644
--- a/protocols/profinet/src/main/resources/protocols/profinet/profinet.dcp.mspec
+++ b/protocols/profinet/src/main/resources/protocols/profinet/profinet.dcp.mspec
@@ -17,6 +17,13 @@
 // under the License.
 //
 
+[type 'EthernetFrame'
+    [simple MacAddress 'destination' ]
+    [simple MacAddress 'source'      ]
+    [simple uint 16    'ethernetType']
+    [simple ProfinetFrame 'payload'  ]
+]
+
 [type 'ProfinetFrame'
     [enum FrameType 'frameType'                ]
     [simple ProfinetData 'frame'  ['frameType']]
diff --git a/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/ProfinetDCPPlcDriver.java b/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/ProfinetDCPPlcDriver.java
new file mode 100644
index 0000000..1c611ed
--- /dev/null
+++ b/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/ProfinetDCPPlcDriver.java
@@ -0,0 +1,103 @@
+/*
+ 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.plc4x.java.profinet.dcp;
+
+import java.util.function.Consumer;
+import java.util.function.ToIntFunction;
+import org.apache.plc4x.java.PlcDriverManager;
+import org.apache.plc4x.java.api.PlcConnection;
+import org.apache.plc4x.java.profinet.dcp.configuration.ProfinetConfiguration;
+import org.apache.plc4x.java.profinet.dcp.field.ProfinetFieldHandler;
+import org.apache.plc4x.java.profinet.dcp.protocol.ProfinetDCPProtocolLogic;
+import org.apache.plc4x.java.profinet.dcp.readwrite.EthernetFrame;
+import org.apache.plc4x.java.profinet.dcp.readwrite.io.EthernetFrameIO;
+import org.apache.plc4x.java.spi.configuration.Configuration;
+import org.apache.plc4x.java.spi.connection.GeneratedDriverBase;
+import org.apache.plc4x.java.spi.connection.ProtocolStackConfigurer;
+import org.apache.plc4x.java.spi.connection.SingleProtocolStackConfigurer;
+
+import io.netty.buffer.ByteBuf;
+
+/**
+ * Proof of concept implementation of Profinet DCP driver.
+ */
+public class ProfinetDCPPlcDriver extends GeneratedDriverBase<EthernetFrame> {
+
+    @Override
+    public String getProtocolCode() {
+        return "profinet-dcp";
+    }
+
+    @Override
+    public String getProtocolName() {
+        return "Profinet Discovery and Configuration Protocol";
+    }
+
+    @Override
+    protected Class<? extends Configuration> getConfigurationType() {
+        return ProfinetConfiguration.class;
+    }
+
+    @Override
+    protected String getDefaultTransport() {
+        return "raw";
+    }
+
+    @Override
+    protected ProfinetFieldHandler getFieldHandler() {
+        return new ProfinetFieldHandler();
+    }
+
+    @Override
+    protected ProtocolStackConfigurer<EthernetFrame> getStackConfigurer() {
+        return SingleProtocolStackConfigurer.builder(EthernetFrame.class, EthernetFrameIO.class)
+            .withProtocol(ProfinetDCPProtocolLogic.class)
+            .withPacketSizeEstimator(ProfinetPacketEstimator.class)
+            .withCorruptPacketRemover(CorruptEthernetFrameRemover.class)
+            .build();
+    }
+
+    public static class ProfinetPacketEstimator implements ToIntFunction<ByteBuf> {
+        @Override
+        public int applyAsInt(ByteBuf value) {
+            if (value.readableBytes() >= 24) {
+                int unsignedShort = value.getUnsignedShort(24);
+                return 26 + unsignedShort;
+            }
+            return -1;
+        }
+    }
+
+    public static class CorruptEthernetFrameRemover implements Consumer<ByteBuf> {
+
+        @Override
+        public void accept(ByteBuf byteBuf) {
+            if (byteBuf.getShort(12) != ProfinetDCPProtocolLogic.PN_DCP) {
+                byteBuf.readBytes(12);
+            }
+        }
+    }
+
+
+    public static void main(String[] args) throws Exception {
+        PlcConnection connection = new PlcDriverManager().getConnection("profinet-dcp:raw://ens2f1");
+        //connection.connect();
+    }
+
+}
diff --git a/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/configuration/ProfinetConfiguration.java b/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/configuration/ProfinetConfiguration.java
new file mode 100644
index 0000000..c9c87c7
--- /dev/null
+++ b/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/configuration/ProfinetConfiguration.java
@@ -0,0 +1,53 @@
+/*
+ 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.plc4x.java.profinet.dcp.configuration;
+
+import org.apache.plc4x.java.spi.configuration.Configuration;
+import org.apache.plc4x.java.transport.rawsocket.RawSocketTransportConfiguration;
+import org.apache.plc4x.java.utils.pcap.netty.handlers.PacketHandler;
+import org.pcap4j.packet.Packet;
+
+public class ProfinetConfiguration implements Configuration, RawSocketTransportConfiguration {
+
+    @Override
+    public boolean getSupportVlans() {
+        return false;
+    }
+
+    @Override
+    public int getDefaultPort() {
+        return -1;
+    }
+
+    @Override
+    public Integer getProtocolId() {
+        return 0x8892;
+    }
+
+    @Override
+    public PacketHandler getPcapPacketHandler() {
+        return new PacketHandler() {
+            @Override
+            public byte[] getData(Packet packet) {
+                // We rely directly on the ethernet frame, so we just need everything.
+                return packet.getRawData();
+            }
+        };
+    }
+}
diff --git a/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/field/ProfinetFieldHandler.java b/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/field/ProfinetFieldHandler.java
new file mode 100644
index 0000000..48ff3a5
--- /dev/null
+++ b/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/field/ProfinetFieldHandler.java
@@ -0,0 +1,36 @@
+/*
+ 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.plc4x.java.profinet.dcp.field;
+
+import org.apache.plc4x.java.api.exceptions.PlcInvalidFieldException;
+import org.apache.plc4x.java.api.model.PlcField;
+import org.apache.plc4x.java.spi.connection.DefaultPlcFieldHandler;
+import org.apache.plc4x.java.spi.connection.PlcFieldHandler;
+
+/**
+ * TODO implement this.
+ */
+public class ProfinetFieldHandler extends DefaultPlcFieldHandler implements PlcFieldHandler {
+
+    @Override
+    public PlcField createField(String fieldQuery) throws PlcInvalidFieldException {
+        return null;
+    }
+
+}
diff --git a/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/protocol/ProfinetDCPProtocolLogic.java b/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/protocol/ProfinetDCPProtocolLogic.java
new file mode 100644
index 0000000..dd170ce
--- /dev/null
+++ b/sandbox/test-java-profinet-driver/src/main/java/org/apache/plc4x/java/profinet/dcp/protocol/ProfinetDCPProtocolLogic.java
@@ -0,0 +1,120 @@
+/*
+ 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.plc4x.java.profinet.dcp.protocol;
+
+import java.time.Duration;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.plc4x.java.profinet.dcp.readwrite.AllSelector;
+import org.apache.plc4x.java.profinet.dcp.readwrite.DCPBlock;
+import org.apache.plc4x.java.profinet.dcp.readwrite.DcpIdentRequestPDU;
+import org.apache.plc4x.java.profinet.dcp.readwrite.DcpIdentResponsePDU;
+import org.apache.plc4x.java.profinet.dcp.readwrite.DeviceProperties;
+import org.apache.plc4x.java.profinet.dcp.readwrite.EthernetFrame;
+import org.apache.plc4x.java.profinet.dcp.readwrite.IP;
+import org.apache.plc4x.java.profinet.dcp.readwrite.IPv4Address;
+import org.apache.plc4x.java.profinet.dcp.readwrite.MacAddress;
+import org.apache.plc4x.java.profinet.dcp.readwrite.ProfinetFrame;
+import org.apache.plc4x.java.profinet.dcp.readwrite.types.DCPBlockOption;
+import org.apache.plc4x.java.profinet.dcp.readwrite.types.DCPServiceID;
+import org.apache.plc4x.java.profinet.dcp.readwrite.types.DCPServiceType;
+import org.apache.plc4x.java.profinet.dcp.readwrite.types.FrameType;
+import org.apache.plc4x.java.spi.ConversationContext;
+import org.apache.plc4x.java.spi.Plc4xProtocolBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Driver logic for handling Profinet-DCP packets.
+ */
+public class ProfinetDCPProtocolLogic extends Plc4xProtocolBase<EthernetFrame> {
+
+    public static MacAddress PROFINET_BROADCAST = createAddress(0x01, 0x0E, 0xCF, 0x00, 0x00, 0x00);
+    public static MacAddress TEST_ADDRESS = createAddress(0x08, 0x00, 0x27, 0x10, 0xFF, 0x10);
+    public static int PN_DCP = 0x8892;
+
+    public static final Duration REQUEST_TIMEOUT = Duration.ofMillis(10000);
+    private final Logger logger = LoggerFactory.getLogger(ProfinetDCPProtocolLogic.class);
+
+    private AtomicInteger invokeId = new AtomicInteger(0);
+
+    @Override
+    public void onConnect(ConversationContext<EthernetFrame> context) {
+        DCPServiceID serviceId = DCPServiceID.IDENTIFY;
+        DCPServiceType serviceType = DCPServiceType.REQUEST;
+        long xid = invokeId.incrementAndGet();
+        int responseDelay = 1000;
+        DCPBlock[] blocks = new DCPBlock[] { new AllSelector(DCPBlockOption.ALL_SELECTOR)};
+        int dcpDataLength = blocks[0].getLengthInBytes();
+
+        DcpIdentRequestPDU requestPDU = new DcpIdentRequestPDU(serviceId, serviceType, xid, responseDelay, dcpDataLength, blocks);
+        EthernetFrame ethernetFrame = new EthernetFrame(PROFINET_BROADCAST, TEST_ADDRESS, PN_DCP,
+            new ProfinetFrame(FrameType.IDENTIFY_MULTICAST_REQUEST, requestPDU));
+
+        // this is broadcast thus reply might come from multiple participants
+        context.sendToWire(ethernetFrame);
+        context.fireConnected();
+    }
+
+    @Override
+    public void onDisconnect(ConversationContext<EthernetFrame> context) {
+        context.fireDisconnected();
+    }
+
+    @Override
+    protected void decode(ConversationContext<EthernetFrame> context, EthernetFrame msg) throws Exception {
+        if (msg.getEthernetType() != PN_DCP) {
+            logger.trace("Discarding unwanted frame type {}", msg.getEthernetType());
+        }
+
+        ProfinetFrame profinetFrame = msg.getPayload();
+        if (profinetFrame.getFrameType() == FrameType.IDENTIFY_RESPONSE) {
+            logger.info("Ident response from Profinet device:");
+            if (profinetFrame.getFrame() instanceof DcpIdentResponsePDU) {
+                DcpIdentResponsePDU response = (DcpIdentResponsePDU) profinetFrame.getFrame();
+                DCPBlock[] blocks = response.getBlocks();
+                for (int index = 0; index < blocks.length; index++) {
+                    DCPBlock block = blocks[index];
+                    if (block instanceof IP) {
+                        IP ip = (IP) block;
+                        logger.info("Device IP: {}, mask: {}, gateway: {}", addressString(ip.getIpAddress()), addressString(ip.getSubnetMask()), addressString(ip.getStandardGateway()));
+                    }
+                    if (block instanceof DeviceProperties) {
+                        DeviceProperties properties = (DeviceProperties) block;
+                        logger.info("Device option: {}, value: {}", properties.getSubOption().name(), properties.getProperties().toPlcValue());
+                    }
+                }
+            } else {
+                logger.error("Unexpected ident response {}", profinetFrame.getFrame().getClass());
+            }
+        }
+    }
+
+    private String addressString(IPv4Address address) {
+        return address.getOctet1() + "." + address.getOctet2() + "." + address.getOctet3() + "." + address.getOctet4();
+    }
+
+    @Override
+    public void close(ConversationContext<EthernetFrame> context) {
+
+    }
+
+    public final static MacAddress createAddress(int octet1, int octet2, int octet3, int octet4, int octet5, int octet6) {
+        return new MacAddress((short) octet1, (short) octet2, (short) octet3, (short) octet4, (short) octet5, (short) octet6);
+    }
+}
diff --git a/sandbox/test-java-profinet-driver/src/main/resources/META-INF/services/org.apache.plc4x.java.api.PlcDriver b/sandbox/test-java-profinet-driver/src/main/resources/META-INF/services/org.apache.plc4x.java.api.PlcDriver
new file mode 100644
index 0000000..f0fa90d
--- /dev/null
+++ b/sandbox/test-java-profinet-driver/src/main/resources/META-INF/services/org.apache.plc4x.java.api.PlcDriver
@@ -0,0 +1,19 @@
+#
+# 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.
+#
+org.apache.plc4x.java.profinet.dcp.ProfinetDCPPlcDriver


[plc4x] 02/02: Implementation of writing part for pcap4j transport.

Posted by ld...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ldywicki pushed a commit to branch feature/profinet
in repository https://gitbox.apache.org/repos/asf/plc4x.git

commit f5874798e01c72a9bff006f54c04af97ba37bf5d
Author: Łukasz Dywicki <lu...@code-house.org>
AuthorDate: Mon May 11 12:57:04 2020 +0200

    Implementation of writing part for pcap4j transport.
---
 .../rawsocket/RawSocketChannelFactory.java         |  9 ++++---
 .../utils/rawsockets/netty/RawSocketChannel.java   | 28 +++++++++++++++-------
 2 files changed, 24 insertions(+), 13 deletions(-)

diff --git a/plc4j/transports/raw-socket/src/main/java/org/apache/plc4x/java/transport/rawsocket/RawSocketChannelFactory.java b/plc4j/transports/raw-socket/src/main/java/org/apache/plc4x/java/transport/rawsocket/RawSocketChannelFactory.java
index 63d7bb5..c8bc6a9 100644
--- a/plc4j/transports/raw-socket/src/main/java/org/apache/plc4x/java/transport/rawsocket/RawSocketChannelFactory.java
+++ b/plc4j/transports/raw-socket/src/main/java/org/apache/plc4x/java/transport/rawsocket/RawSocketChannelFactory.java
@@ -58,12 +58,11 @@ public class RawSocketChannelFactory extends NettyChannelFactory implements HasC
 
     @Override
     public void configureBootstrap(Bootstrap bootstrap) {
-        if(configuration != null) {
+        if (configuration != null) {
             logger.info("Configuring Bootstrap with {}", configuration);
-            /*bootstrap.option(RawSocketChannelOption.PORT, configuration.getDefaultPort());
-            bootstrap.option(RawSocketChannelOption.PROTOCOL_ID, configuration.getProtocolId());
-            bootstrap.option(RawSocketChannelOption.SPEED_FACTOR, configuration.getReplaySpeedFactor());*/
-            if(configuration.getPcapPacketHandler() != null) {
+            bootstrap.option(PcapChannelOption.PORT, configuration.getDefaultPort());
+            bootstrap.option(PcapChannelOption.PROTOCOL_ID, configuration.getProtocolId());
+            if (configuration.getPcapPacketHandler() != null) {
                 bootstrap.option(PcapChannelOption.PACKET_HANDLER, configuration.getPcapPacketHandler());
             }
         }
diff --git a/plc4j/utils/raw-sockets/src/main/java/org/apache/plc4x/java/utils/rawsockets/netty/RawSocketChannel.java b/plc4j/utils/raw-sockets/src/main/java/org/apache/plc4x/java/utils/rawsockets/netty/RawSocketChannel.java
index b3391f7..098c21a 100644
--- a/plc4j/utils/raw-sockets/src/main/java/org/apache/plc4x/java/utils/rawsockets/netty/RawSocketChannel.java
+++ b/plc4j/utils/raw-sockets/src/main/java/org/apache/plc4x/java/utils/rawsockets/netty/RawSocketChannel.java
@@ -96,7 +96,7 @@ public class RawSocketChannel extends OioByteStreamChannel {
         PcapNetworkInterface nif = Pcaps.getDevByName(deviceName);
         handle = nif.openLive(65535, PcapNetworkInterface.PromiscuousMode.PROMISCUOUS, 10);
         if(logger.isDebugEnabled()) {
-            logger.debug(String.format("Listening on device %s", deviceName));
+            logger.debug("Listening on device {}", deviceName);
         }
 
         // If the address allows fine tuning which packets to process, set a filter to reduce the load.
@@ -135,7 +135,7 @@ public class RawSocketChannel extends OioByteStreamChannel {
         // be able to actually send data. The PcapInputStream simply acts as a
         // breaking point if no packets are coming in and the read operation would
         // simply block indefinitely.
-        activate(new PcapInputStream(buffer), new DiscardingOutputStream());
+        activate(new PcapInputStream(buffer), new PcapOutputStream(handle));
     }
 
     @Override
@@ -199,18 +199,28 @@ public class RawSocketChannel extends OioByteStreamChannel {
     }
 
     /**
-     * This output stream simply discards anything it should send.
+     * This output stream simply pass whole buffer to pcap library hopping it will be
+     * sent over interface.
      */
-    private static class DiscardingOutputStream extends OutputStream {
+    private static class PcapOutputStream extends OutputStream {
+        private final PcapHandle handle;
+
+        public PcapOutputStream(PcapHandle handle) {
+            this.handle = handle;
+        }
+
         @Override
         public void write(int b) throws IOException {
-            // discard
-            logger.debug("Discarding {}", b);
+            throw new IOException("Appending single bytes is not permitted. Use write(byte[], int, int)");
         }
 
         @Override
-        public void write(byte[] b, int off, int len) {
-            logger.debug("Discarding {}", b);
+        public void write(byte[] b, int off, int len) throws IOException {
+            try {
+                handle.sendPacket(b);
+            } catch (NotOpenException | PcapNativeException e) {
+                throw new IOException(e);
+            }
         }
     }
 
@@ -245,6 +255,7 @@ public class RawSocketChannel extends OioByteStreamChannel {
             }
             throw new SocketTimeoutException();
         }
+
     }
 
     /**
@@ -262,4 +273,5 @@ public class RawSocketChannel extends OioByteStreamChannel {
             }
         }
     }
+
 }