Added: incubator/kafka/trunk/clients/clojure/src/kafka/buffer.clj
URL: http://svn.apache.org/viewvc/incubator/kafka/trunk/clients/clojure/src/kafka/buffer.clj?rev=1152970&view=auto
==============================================================================
--- incubator/kafka/trunk/clients/clojure/src/kafka/buffer.clj (added)
+++ incubator/kafka/trunk/clients/clojure/src/kafka/buffer.clj Mon Aug 1 23:41:24 2011
@@ -0,0 +1,175 @@
+(ns #^{:doc "Wrapper around ByteBuffer,
+ provides a DSL to model byte messages."}
+ kafka.buffer
+ (:import (java.nio ByteBuffer)
+ (java.nio.channels SocketChannel)))
+
+(def #^{:doc "Buffer stack bind in with-buffer."}
+ *buf* [])
+
+(def #^{:doc "Number of attempts to read a complete buffer from channel."}
+ *channel-read-count* 5)
+
+;
+; Main buffer functions
+;
+
+(defn buffer
+ "Creates a new ByteBuffer of argument size."
+ [^int size]
+ (ByteBuffer/allocate size))
+
+(defn ^ByteBuffer top
+ "Returns top buffer from *buf* stack."
+ []
+ (peek *buf*))
+
+(defn flip
+ []
+ (.flip (top)))
+
+(defn rewind
+ []
+ (.rewind (top)))
+
+(defn clear
+ []
+ (.clear (top)))
+
+(defn has-remaining
+ []
+ (.hasRemaining (top)))
+
+;
+; Write to buffer
+;
+
+(defprotocol Put
+ "Put protocol defines a generic buffer put method."
+ (put [this]))
+
+(extend-type Byte
+ Put
+ (put [this] (.put (top) this)))
+
+(extend-type Integer
+ Put
+ (put [this] (.putInt (top) this)))
+
+(extend-type Short
+ Put
+ (put [this] (.putShort (top) this)))
+
+(extend-type Long
+ Put
+ (put [this] (.putLong (top) this)))
+
+(extend-type String
+ Put
+ (put [this] (.put (top) (.getBytes this "UTF-8"))))
+
+(extend-type (class (byte-array 0))
+ Put
+ (put [this] (.put (top) ^bytes this)))
+
+(extend-type clojure.lang.IPersistentCollection
+ Put
+ (put [this] (doseq [e this] (put e))))
+
+(defmacro length-encoded
+ [type & body]
+ `(with-buffer (.slice (top))
+ (put (~type 0))
+ (let [^ByteBuffer this# (top)
+ ^ByteBuffer parent# (peek (pop *buf*))
+ type-size# (.position this#)]
+ ~@body
+ (let [size# (.position this#)]
+ (.rewind this#)
+ (put (~type (- size# type-size#)))
+ (.position parent# (+ (.position parent#) size#))))))
+
+(defmacro with-put
+ [size f & body]
+ `(with-buffer (.slice (top))
+ (put (byte-array ~size))
+ ~@body
+ (let [^ByteBuffer this# (top)
+ ^ByteBuffer parent# (peek (pop *buf*))
+ pos# (.position this#)
+ ba# (byte-array (- pos# ~size))]
+ (doto this# (.rewind) (.get (byte-array ~size)) (.get ba#))
+ (.rewind this#)
+ (put (~f ba#))
+ (.position parent# (+ (.position parent#) pos#)))))
+
+;
+; Read from buffer
+;
+
+(defn get-byte
+ []
+ (.get (top)))
+
+(defn get-short
+ []
+ (.getShort (top)))
+
+(defn get-int
+ []
+ (.getInt (top)))
+
+(defn get-long
+ []
+ (.getLong (top)))
+
+(defn get-array
+ "Reads byte array of argument length from buffer."
+ [^int length]
+ (let [ba (byte-array length)]
+ (.get (top) ba)
+ ba))
+
+(defn get-string
+ "Reads string of argument length from buffer."
+ [^int length]
+ (let [ba (byte-array length)]
+ (.get (top) ba)
+ (String. ba "UTF-8")))
+
+;
+; Util functions and macros
+;
+
+(defmacro with-buffer
+ "Evaluates body in the context of the buffer."
+ [buffer & body]
+ `(binding [*buf* (conj *buf* ~buffer)]
+ ~@body))
+
+(defn read-from
+ "Reads from channel to the underlying top buffer.
+ Throws ConnectException if channel is closed."
+ [^SocketChannel channel]
+ (let [size (.read channel (top))]
+ (if (< size 0)
+ (throw (java.net.ConnectException. "Channel closed?"))
+ size)))
+
+(defn read-completely-from
+ "Read the complete top buffer from the channel."
+ [^SocketChannel channel]
+ (loop [t *channel-read-count* size 0]
+ (let [s (read-from channel)]
+ (cond
+ (< t 0)
+ (throw (Exception. "Unable to read complete buffer from channel."))
+ (has-remaining)
+ (recur (dec t) (+ size s))
+ :else size))))
+
+(defn write-to
+ "Writes underlying top buffer to channel."
+ [^SocketChannel channel]
+ (.write channel (top)))
+
Added: incubator/kafka/trunk/clients/clojure/src/kafka/example.clj
URL: http://svn.apache.org/viewvc/incubator/kafka/trunk/clients/clojure/src/kafka/example.clj?rev=1152970&view=auto
==============================================================================
--- incubator/kafka/trunk/clients/clojure/src/kafka/example.clj (added)
+++ incubator/kafka/trunk/clients/clojure/src/kafka/example.clj Mon Aug 1 23:41:24 2011
@@ -0,0 +1,38 @@
+(ns #^{:doc "Producer/Consumer example."}
+ kafka.example
+ (:use (clojure.contrib logging)
+ (kafka types kafka print)))
+
+(defmacro thread
+ "Executes body in a thread, logs exceptions."
+ [ & body]
+ `(future
+ (try
+ ~@body
+ (catch Exception e#
+ (error "Exception." e#)))))
+
+(defn start-consumer
+ []
+ (thread
+ (with-open [c (consumer "localhost" 9092)]
+ (doseq [m (consume-seq c "test" 0 {:blocking true})]
+ (println "Consumed <-- " m)))
+ (println "Finished consuming.")))
+
+(defn start-producer
+ []
+ (thread
+ (with-open [p (producer "localhost" 9092)]
+ (doseq [i (range 1 20)]
+ (let [m (str "Message " i)]
+ (produce p "test" 0 m)
+ (println "Produced --> " m)
+ (Thread/sleep 1000))))
+ (println "Finished producing.")))
+
+(defn run
+ []
+ (start-consumer)
+ (start-producer))
+
Added: incubator/kafka/trunk/clients/clojure/src/kafka/kafka.clj
URL: http://svn.apache.org/viewvc/incubator/kafka/trunk/clients/clojure/src/kafka/kafka.clj?rev=1152970&view=auto
==============================================================================
--- incubator/kafka/trunk/clients/clojure/src/kafka/kafka.clj (added)
+++ incubator/kafka/trunk/clients/clojure/src/kafka/kafka.clj Mon Aug 1 23:41:24 2011
@@ -0,0 +1,267 @@
+(ns #^{:doc "Core kafka-clj module,
+ provides producer and consumer factories."}
+ kafka.kafka
+ (:use (kafka types buffer)
+ (clojure.contrib logging))
+ (:import (kafka.types Message)
+ (java.nio.channels SocketChannel)
+ (java.net Socket InetSocketAddress)
+ (java.util.zip CRC32)))
+
+;
+; Utils
+;
+
+(defn- crc32-int
+ "CRC for byte array."
+ [^bytes ba]
+ (let [crc (doto (CRC32.) (.update ba))
+ lv (.getValue crc)]
+ (.intValue (bit-and lv 0xffffffff))))
+
+(defn- new-channel
+ "Create and setup a new channel for a host name, port and options.
+ Supported options:
+ :receive-buffer-size - receive socket buffer size, default 65536.
+ :send-buffer-size - send socket buffer size, default 65536.
+ :socket-timeout - socket timeout."
+ [^String host ^Integer port opts]
+ (let [receive-buf-size (or (:receive-buffer-size opts) 65536)
+ send-buf-size (or (:send-buffer-size opts) 65536)
+ so-timeout (or (:socket-timeout opts) 60000)
+ ch (SocketChannel/open)]
+ (doto (.socket ch)
+ (.setReceiveBufferSize receive-buf-size)
+ (.setSendBufferSize send-buf-size)
+ (.setSoTimeout so-timeout))
+ (doto ch
+ (.configureBlocking true)
+ (.connect (InetSocketAddress. host port)))))
+
+(defn- close-channel
+ "Close the channel."
+ [^SocketChannel channel]
+ (.close channel)
+ (.close (.socket channel)))
+
+(defn- response-size
+ "Read first four bytes from channel as an integer."
+ [channel]
+ (with-buffer (buffer 4)
+ (read-completely-from channel)
+ (flip)
+ (get-int)))
+
+(defmacro with-error-code
+ "Convenience response error code check."
+ [request & body]
+ `(let [error-code# (get-short)] ; error code
+ (if (not= error-code# 0)
+ (error (str "Request " ~request " returned error code: " error-code# "."))
+ ~@body)))
+
+;
+; Producer
+;
+
+(defn- send-message
+ "Send messages."
+ [channel topic partition messages opts]
+ (let [size (or (:send-buffer-size opts) 65536)]
+ (with-buffer (buffer size)
+ (length-encoded int ; request size
+ (put (short 0)) ; request type
+ (length-encoded short ; topic size
+ (put topic)) ; topic
+ (put (int partition)) ; partition
+ (length-encoded int ; messages size
+ (doseq [m messages]
+ (let [^Message pm (pack m)]
+ (length-encoded int ; message size
+ (put (byte 0)) ; magic
+ (with-put 4 crc32-int ; crc
+ (put (.message pm)))))))) ; message
+ (flip)
+ (write-to channel))))
+
+(defn producer
+ "Producer factory. See new-channel for list of supported options."
+ [host port & [opts]]
+ (let [channel (new-channel host port opts)]
+ (reify Producer
+ (produce [this topic partition messages]
+ (let [msg (if (sequential? messages) messages [messages])]
+ (send-message channel topic partition msg opts)))
+ (close [this]
+ (close-channel channel)))))
+
+;
+; Consumer
+;
+
+; Offset
+
+(defn- offset-fetch-request
+ "Fetch offsets request."
+ [channel topic partition time max-offsets]
+ (let [size (+ 4 2 2 (count topic) 4 8 4)]
+ (with-buffer (buffer size)
+ (length-encoded int ; request size
+ (put (short 4)) ; request type
+ (length-encoded short ; topic size
+ (put topic)) ; topic
+ (put (int partition)) ; partition
+ (put (long time)) ; time
+ (put (int max-offsets))) ; max-offsets
+ (flip)
+ (write-to channel))))
+
+(defn- fetch-offsets
+ "Fetch offsets as an integer sequence."
+ [channel topic partition time max-offsets]
+ (offset-fetch-request channel topic partition time max-offsets)
+ (let [rsp-size (response-size channel)]
+ (with-buffer (buffer rsp-size)
+ (read-completely-from channel)
+ (flip)
+ (with-error-code "Fetch-Offsets"
+ (loop [c (get-int) res []]
+ (if (> c 0)
+ (recur (dec c) (conj res (get-long)))
+ (doall res)))))))
+
+; Messages
+
+(defn- message-fetch-request
+ "Fetch messages request."
+ [channel topic partition offset max-size]
+ (let [size (+ 4 2 2 (count topic) 4 8 4)]
+ (with-buffer (buffer size)
+ (length-encoded int ; request size
+ (put (short 1)) ; request type
+ (length-encoded short ; topic size
+ (put topic)) ; topic
+ (put (int partition)) ; partition
+ (put (long offset)) ; offset
+ (put (int max-size))) ; max size
+ (flip)
+ (write-to channel))))
+
+(defn- read-response
+ "Read response from buffer. Returns a pair [new offset, messages sequence]."
+ [offset]
+ (with-error-code "Fetch-Messages"
+ (loop [off offset msg []]
+ (if (has-remaining)
+ (let [size (get-int) ; message size
+ magic (get-byte) ; magic
+ crc (get-int) ; crc
+ message (get-array (- size 5))]
+ (recur (+ off size 4) (conj msg (unpack (Message. message)))))
+ [off (doall msg)]))))
+
+(defn- fetch-messages
+ "Message fetch, returns a pair [new offset, messages sequence]."
+ [channel topic partition offset max-size]
+ (message-fetch-request channel topic partition offset max-size)
+ (let [rsp-size (response-size channel)]
+ (with-buffer (buffer rsp-size)
+ (read-completely-from channel)
+ (flip)
+ (read-response offset))))
+
+; Consumer sequence
+
+(defn- seq-fetch
+ "Non-blocking fetch function used by consumer sequence."
+ [channel topic partition opts]
+ (let [max-size (or (:max-size opts) 1000000)]
+ (fn [offset]
+ (fetch-messages channel topic partition offset max-size))))
+
+(defn- blocking-seq-fetch
+ "Blocking fetch function used by consumer sequence."
+ [channel topic partition opts]
+ (let [repeat-count (or (:repeat-count opts) 10)
+ repeat-timeout (or (:repeat-timeout opts) 1000)
+ max-size (or (:max-size opts) 1000000)]
+ (fn [offset]
+ (loop [c repeat-count]
+ (if (> c 0)
+ (let [rs (fetch-messages channel topic partition offset max-size)]
+ (if (or (nil? rs) (= offset (first rs)))
+ (do
+ (Thread/sleep repeat-timeout)
+ (recur (dec c)))
+ (doall rs)))
+ (debug "Stopping blocking seq fetch."))))))
+
+(defn- fetch-queue
+ [offset queue fetch-fn]
+ (if (empty? @queue)
+ (let [[new-offset msg] (fetch-fn @offset)]
+ (when new-offset
+ (debug (str "Fetched " (count msg) " messages:"))
+ (debug (str "New offset " new-offset "."))
+ (swap! queue #(reduce conj % (reverse msg)))
+ (reset! offset new-offset)))))
+
+(defn- consumer-seq
+ "Sequence constructor."
+ [offset fetch-fn]
+ (let [offset (atom offset)
+ queue (atom (seq []))]
+ (reify
+ clojure.lang.IPersistentCollection
+ (seq [this] this)
+ (cons [this _] (throw (Exception. "cons not supported for consumer sequence.")))
+ (empty [this] nil)
+ (equiv [this o]
+ (fatal "Implement equiv for consumer seq!")
+ false)
+ clojure.lang.ISeq
+ (first [this]
+ (fetch-queue offset queue fetch-fn)
+ (first @queue))
+ (next [this]
+ (swap! queue rest)
+ (fetch-queue offset queue fetch-fn)
+ (if (not (empty? @queue)) this))
+ (more [this]
+ (swap! queue rest)
+ (fetch-queue offset queue fetch-fn)
+ (if (empty? @queue) (empty) this))
+ Object
+ (toString [this]
+ (str "ConsumerQueue")))))
+
+; Consumer factory
+
+(defn consumer
+ "Consumer factory. See new-channel for list of supported options."
+ [host port & [opts]]
+ (let [channel (new-channel host port opts)]
+ (reify Consumer
+ (consume [this topic partition offset max-size]
+ (fetch-messages channel topic partition offset max-size))
+
+ (offsets [this topic partition time max-offsets]
+ (fetch-offsets channel topic partition time max-offsets))
+
+ (consume-seq [this topic partition]
+ (let [[offset] (fetch-offsets channel topic partition -1 1)]
+ (debug (str "Initializing last offset to " offset "."))
+ (consumer-seq (or offset 0) (seq-fetch channel topic partition opts))))
+
+ (consume-seq [this topic partition opts]
+ (let [[offset] (or (:offset opts)
+ (fetch-offsets channel topic partition -1 1))
+ fetch-fn (if (:blocking opts)
+ (blocking-seq-fetch channel topic partition opts)
+ (seq-fetch channel topic partition opts))]
+ (debug (str "Initializing last offset to " offset "."))
+ (consumer-seq (or offset 0) fetch-fn)))
+
+ (close [this]
+ (close-channel channel)))))
+
Added: incubator/kafka/trunk/clients/clojure/src/kafka/print.clj
URL: http://svn.apache.org/viewvc/incubator/kafka/trunk/clients/clojure/src/kafka/print.clj?rev=1152970&view=auto
==============================================================================
--- incubator/kafka/trunk/clients/clojure/src/kafka/print.clj (added)
+++ incubator/kafka/trunk/clients/clojure/src/kafka/print.clj Mon Aug 1 23:41:24 2011
@@ -0,0 +1,22 @@
+(ns #^{:doc "Basic Clojure print-dup -> read-string message serialization."}
+ kafka.print
+ (:use kafka.types)
+ (:import (kafka.types Message)))
+
+(extend-type Object
+ Pack
+ (pack [this]
+ (let [^String st (with-out-str (print-dup this *out*))]
+ (kafka.types.Message. (.getBytes st "UTF-8")))))
+
+(extend-type Message
+ Unpack
+ (unpack [this]
+ (let [^bytes ba (.message this)
+ msg (String. ba "UTF-8")]
+ (if (not (empty? msg))
+ (try
+ (read-string msg)
+ (catch Exception e
+ (println "Invalid expression " msg)))))))
+
Added: incubator/kafka/trunk/clients/clojure/src/kafka/serializable.clj
URL: http://svn.apache.org/viewvc/incubator/kafka/trunk/clients/clojure/src/kafka/serializable.clj?rev=1152970&view=auto
==============================================================================
--- incubator/kafka/trunk/clients/clojure/src/kafka/serializable.clj (added)
+++ incubator/kafka/trunk/clients/clojure/src/kafka/serializable.clj Mon Aug 1 23:41:24 2011
@@ -0,0 +1,22 @@
+(ns #^{:doc "Serialization for all Java Serializable objects."}
+ kafka.serializable
+ (:use kafka.types)
+ (:import (kafka.types Message)
+ (java.io Serializable
+ ObjectOutputStream ByteArrayOutputStream
+ ObjectInputStream ByteArrayInputStream)))
+
+(extend-type Serializable
+ Pack
+ (pack [this]
+ (let [bas (ByteArrayOutputStream.)]
+ (with-open [oos (ObjectOutputStream. bas)]
+ (.writeObject oos this))
+ (kafka.types.Message. (.toByteArray bas)))))
+
+(extend-type Message
+ Unpack
+ (unpack [this]
+ (with-open [ois (ObjectInputStream. (ByteArrayInputStream. (.message this)))]
+ (.readObject ois))))
+
Added: incubator/kafka/trunk/clients/clojure/src/kafka/types.clj
URL: http://svn.apache.org/viewvc/incubator/kafka/trunk/clients/clojure/src/kafka/types.clj?rev=1152970&view=auto
==============================================================================
--- incubator/kafka/trunk/clients/clojure/src/kafka/types.clj (added)
+++ incubator/kafka/trunk/clients/clojure/src/kafka/types.clj Mon Aug 1 23:41:24 2011
@@ -0,0 +1,28 @@
+(ns #^{:doc "Base kafka-clj types."}
+ kafka.types)
+
+(deftype #^{:doc "Message type, a wrapper around a byte array."}
+ Message [^bytes message])
+
+(defprotocol Pack
+ "Pack protocol converts an object to a Message."
+ (pack [this] "Convert object to a Message."))
+
+(defprotocol Unpack
+ "Unpack protocol, reads an object from a Message."
+ (unpack [^Message this] "Read an object from the message."))
+
+(defprotocol Producer
+ "Producer protocol."
+ (produce [this topic partition messages] "Send message[s] for a topic to a partition.")
+ (close [this] "Closes the producer, socket and channel."))
+
+(defprotocol Consumer
+ "Consumer protocol."
+ (consume [this topic partition offset max-size] "Fetch messages. Returns a pair [last-offset, message sequence]")
+ (offsets [this topic partition time max-offsets] "Query offsets. Returns offsets seq.")
+
+ (consume-seq [this topic partition]
+ [this topic partition opts] "Creates a sequence over the consumer.")
+ (close [this] "Close the consumer, socket and channel."))
+
Added: incubator/kafka/trunk/clients/clojure/test/kafka/buffer_test.clj
URL: http://svn.apache.org/viewvc/incubator/kafka/trunk/clients/clojure/test/kafka/buffer_test.clj?rev=1152970&view=auto
==============================================================================
--- incubator/kafka/trunk/clients/clojure/test/kafka/buffer_test.clj (added)
+++ incubator/kafka/trunk/clients/clojure/test/kafka/buffer_test.clj Mon Aug 1 23:41:24 2011
@@ -0,0 +1,46 @@
+(ns kafka.buffer-test
+ (:use (kafka buffer)
+ clojure.test))
+
+(deftest test-put-get
+ (with-buffer (buffer 64)
+ (put (byte 5))
+ (put (short 10))
+ (put (int 20))
+ (put (long 40))
+ (put "test")
+ (put (byte-array 3 [(byte 1) (byte 2) (byte 3)]))
+ (flip)
+
+ (is (= (get-byte) (byte 5)))
+ (is (= (get-short) (short 10)))
+ (is (= (get-int) (int 20)))
+ (is (= (get-long) (long 40)))
+ (is (= (get-string 4) "test"))
+ (let [ba (get-array 3)]
+ (is (= (nth ba 0) (byte 1)))
+ (is (= (nth ba 1) (byte 2)))
+ (is (= (nth ba 2) (byte 3))))))
+
+(deftest test-with-put
+ (with-buffer (buffer 64)
+ (with-put 4 count
+ (put "test 1"))
+ (flip)
+
+ (is (= (get-int) (int 6)))
+ (is (= (get-string 6) "test 1"))))
+
+(deftest test-length-encoded
+ (with-buffer (buffer 64)
+ (length-encoded short
+ (put "test 1"))
+ (length-encoded int
+ (put "test 2"))
+ (flip)
+
+ (is (= (get-short) (short 6)))
+ (is (= (get-string 6) "test 1"))
+ (is (= (get-int) (int 6)))
+ (is (= (get-string 6) "test 2"))))
+
Added: incubator/kafka/trunk/clients/clojure/test/kafka/print_test.clj
URL: http://svn.apache.org/viewvc/incubator/kafka/trunk/clients/clojure/test/kafka/print_test.clj?rev=1152970&view=auto
==============================================================================
--- incubator/kafka/trunk/clients/clojure/test/kafka/print_test.clj (added)
+++ incubator/kafka/trunk/clients/clojure/test/kafka/print_test.clj Mon Aug 1 23:41:24 2011
@@ -0,0 +1,12 @@
+(ns kafka.print-test
+ (:use (kafka types print)
+ clojure.test))
+
+(deftest test-pack-unpack
+ (is (= "test" (unpack (pack "test"))))
+ (is (= 123 (unpack (pack 123))))
+ (is (= true (unpack (pack true))))
+ (is (= [1 2 3] (unpack (pack [1 2 3]))))
+ (is (= {:a 1} (unpack (pack {:a 1}))))
+ (is (= '(+ 1 2 3) (unpack (pack '(+ 1 2 3))))))
+
Added: incubator/kafka/trunk/clients/clojure/test/kafka/serializable_test.clj
URL: http://svn.apache.org/viewvc/incubator/kafka/trunk/clients/clojure/test/kafka/serializable_test.clj?rev=1152970&view=auto
==============================================================================
--- incubator/kafka/trunk/clients/clojure/test/kafka/serializable_test.clj (added)
+++ incubator/kafka/trunk/clients/clojure/test/kafka/serializable_test.clj Mon Aug 1 23:41:24 2011
@@ -0,0 +1,14 @@
+(ns kafka.serializable-test
+ (:use (kafka types serializable)
+ clojure.test))
+
+(deftest test-pack-unpack
+ (is (= "test" (unpack (pack "test"))))
+ (is (= 123 (unpack (pack 123))))
+ (is (= true (unpack (pack true))))
+ (is (= [1 2 3] (unpack (pack [1 2 3]))))
+ (is (= {:a 1} (unpack (pack {:a 1}))))
+ (is (= '(+ 1 2 3) (unpack (pack '(+ 1 2 3)))))
+ (let [now (java.util.Date.)]
+ (is (= now (unpack (pack now))))))
+
Added: incubator/kafka/trunk/clients/cpp/LICENSE
URL: http://svn.apache.org/viewvc/incubator/kafka/trunk/clients/cpp/LICENSE?rev=1152970&view=auto
==============================================================================
--- incubator/kafka/trunk/clients/cpp/LICENSE (added)
+++ incubator/kafka/trunk/clients/cpp/LICENSE Mon Aug 1 23:41:24 2011
@@ -0,0 +1,203 @@
+
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+END OF TERMS AND CONDITIONS
+
+APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+Copyright 2011 LinkedIn
+
+Licensed 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.
+
Added: incubator/kafka/trunk/clients/cpp/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/kafka/trunk/clients/cpp/Makefile.am?rev=1152970&view=auto
==============================================================================
--- incubator/kafka/trunk/clients/cpp/Makefile.am (added)
+++ incubator/kafka/trunk/clients/cpp/Makefile.am Mon Aug 1 23:41:24 2011
@@ -0,0 +1,51 @@
+## LibKafkaConect
+## A C++ shared libray for connecting to Kafka
+
+#
+# Warning this is the first time I've made a configure.ac/Makefile.am thing
+# Please improve it as I have no idea what I am doing
+# @benjamg
+#
+
+ACLOCAL_AMFLAGS = -I build-aux/m4 ${ACLOCAL_FLAGS}
+AM_CPPFLAGS = $(DEPS_CFLAGS)
+EXAMPLE_LIBS = -lboost_system -lboost_thread -lkafkaconnect
+
+#
+# Shared Library
+#
+
+lib_LTLIBRARIES = libkafkaconnect.la
+
+libkafkaconnect_la_SOURCES = src/producer.cpp
+libkafkaconnect_la_LDFLAGS = -version-info $(KAFKACONNECT_VERSION)
+
+kafkaconnect_includedir = $(includedir)/kafkaconnect
+kafkaconnect_include_HEADERS = src/producer.hpp \
+ src/encoder.hpp \
+ src/encoder_helper.hpp
+
+#
+# Examples
+#
+
+noinst_PROGRAMS = producer
+
+producer_SOURCES = src/example.cpp
+producer_LDADD = $(DEPS_LIBS) $(EXAMPLE_LIBS)
+
+#
+# Tests
+#
+
+check_PROGRAMS = tests/encoder_helper tests/encoder tests/producer
+TESTS = tests/encoder_helper tests/encoder tests/producer
+
+tests_encoder_helper_SOURCES = src/tests/encoder_helper_tests.cpp
+tests_encoder_helper_LDADD = $(DEPS_LIBS) $(EXAMPLE_LIBS) -lboost_unit_test_framework
+
+tests_encoder_SOURCES = src/tests/encoder_tests.cpp
+tests_encoder_LDADD = $(DEPS_LIBS) $(EXAMPLE_LIBS) -lboost_unit_test_framework
+
+tests_producer_SOURCES = src/tests/producer_tests.cpp
+tests_producer_LDADD = $(DEPS_LIBS) $(EXAMPLE_LIBS) -lboost_unit_test_framework
Added: incubator/kafka/trunk/clients/cpp/Makefile.in
URL: http://svn.apache.org/viewvc/incubator/kafka/trunk/clients/cpp/Makefile.in?rev=1152970&view=auto
==============================================================================
--- incubator/kafka/trunk/clients/cpp/Makefile.in (added)
+++ incubator/kafka/trunk/clients/cpp/Makefile.in Mon Aug 1 23:41:24 2011
@@ -0,0 +1,946 @@
+# Makefile.in generated by automake 1.11.1 from Makefile.am.
+# @configure_input@
+
+# Copyright (C) 1994, 1995, 1996, 1997, 1998, 1999, 2000, 2001, 2002,
+# 2003, 2004, 2005, 2006, 2007, 2008, 2009 Free Software Foundation,
+# Inc.
+# This Makefile.in is free software; the Free Software Foundation
+# gives unlimited permission to copy and/or distribute it,
+# with or without modifications, as long as this notice is preserved.
+
+# This program is distributed in the hope that it will be useful,
+# but WITHOUT ANY WARRANTY, to the extent permitted by law; without
+# even the implied warranty of MERCHANTABILITY or FITNESS FOR A
+# PARTICULAR PURPOSE.
+
+@SET_MAKE@
+
+#
+# Warning this is the first time I've made a configure.ac/Makefile.am thing
+# Please improve it as I have no idea what I am doing
+# @benjamg
+#
+
+
+
+VPATH = @srcdir@
+pkgdatadir = $(datadir)/@PACKAGE@
+pkgincludedir = $(includedir)/@PACKAGE@
+pkglibdir = $(libdir)/@PACKAGE@
+pkglibexecdir = $(libexecdir)/@PACKAGE@
+am__cd = CDPATH="$${ZSH_VERSION+.}$(PATH_SEPARATOR)" && cd
+install_sh_DATA = $(install_sh) -c -m 644
+install_sh_PROGRAM = $(install_sh) -c
+install_sh_SCRIPT = $(install_sh) -c
+INSTALL_HEADER = $(INSTALL_DATA)
+transform = $(program_transform_name)
+NORMAL_INSTALL = :
+PRE_INSTALL = :
+POST_INSTALL = :
+NORMAL_UNINSTALL = :
+PRE_UNINSTALL = :
+POST_UNINSTALL = :
+build_triplet = @build@
+host_triplet = @host@
+noinst_PROGRAMS = producer$(EXEEXT)
+check_PROGRAMS = tests/encoder_helper$(EXEEXT) tests/encoder$(EXEEXT) \
+ tests/producer$(EXEEXT)
+TESTS = tests/encoder_helper$(EXEEXT) tests/encoder$(EXEEXT) \
+ tests/producer$(EXEEXT)
+subdir = .
+DIST_COMMON = $(am__configure_deps) $(kafkaconnect_include_HEADERS) \
+ $(srcdir)/Makefile.am $(srcdir)/Makefile.in \
+ $(top_srcdir)/configure build-aux/config.guess \
+ build-aux/config.sub build-aux/depcomp build-aux/install-sh \
+ build-aux/ltmain.sh build-aux/missing
+ACLOCAL_M4 = $(top_srcdir)/aclocal.m4
+am__aclocal_m4_deps = $(top_srcdir)/build-aux/m4/libtool.m4 \
+ $(top_srcdir)/build-aux/m4/ltoptions.m4 \
+ $(top_srcdir)/build-aux/m4/ltsugar.m4 \
+ $(top_srcdir)/build-aux/m4/ltversion.m4 \
+ $(top_srcdir)/build-aux/m4/lt~obsolete.m4 \
+ $(top_srcdir)/configure.ac
+am__configure_deps = $(am__aclocal_m4_deps) $(CONFIGURE_DEPENDENCIES) \
+ $(ACLOCAL_M4)
+am__CONFIG_DISTCLEAN_FILES = config.status config.cache config.log \
+ configure.lineno config.status.lineno
+mkinstalldirs = $(install_sh) -d
+CONFIG_CLEAN_FILES =
+CONFIG_CLEAN_VPATH_FILES =
+am__vpath_adj_setup = srcdirstrip=`echo "$(srcdir)" | sed 's|.|.|g'`;
+am__vpath_adj = case $$p in \
+ $(srcdir)/*) f=`echo "$$p" | sed "s|^$$srcdirstrip/||"`;; \
+ *) f=$$p;; \
+ esac;
+am__strip_dir = f=`echo $$p | sed -e 's|^.*/||'`;
+am__install_max = 40
+am__nobase_strip_setup = \
+ srcdirstrip=`echo "$(srcdir)" | sed 's/[].[^$$\\*|]/\\\\&/g'`
+am__nobase_strip = \
+ for p in $$list; do echo "$$p"; done | sed -e "s|$$srcdirstrip/||"
+am__nobase_list = $(am__nobase_strip_setup); \
+ for p in $$list; do echo "$$p $$p"; done | \
+ sed "s| $$srcdirstrip/| |;"' / .*\//!s/ .*/ ./; s,\( .*\)/[^/]*$$,\1,' | \
+ $(AWK) 'BEGIN { files["."] = "" } { files[$$2] = files[$$2] " " $$1; \
+ if (++n[$$2] == $(am__install_max)) \
+ { print $$2, files[$$2]; n[$$2] = 0; files[$$2] = "" } } \
+ END { for (dir in files) print dir, files[dir] }'
+am__base_list = \
+ sed '$$!N;$$!N;$$!N;$$!N;$$!N;$$!N;$$!N;s/\n/ /g' | \
+ sed '$$!N;$$!N;$$!N;$$!N;s/\n/ /g'
+am__installdirs = "$(DESTDIR)$(libdir)" \
+ "$(DESTDIR)$(kafkaconnect_includedir)"
+LTLIBRARIES = $(lib_LTLIBRARIES)
+libkafkaconnect_la_LIBADD =
+am_libkafkaconnect_la_OBJECTS = producer.lo
+libkafkaconnect_la_OBJECTS = $(am_libkafkaconnect_la_OBJECTS)
+libkafkaconnect_la_LINK = $(LIBTOOL) --tag=CXX $(AM_LIBTOOLFLAGS) \
+ $(LIBTOOLFLAGS) --mode=link $(CXXLD) $(AM_CXXFLAGS) \
+ $(CXXFLAGS) $(libkafkaconnect_la_LDFLAGS) $(LDFLAGS) -o $@
+PROGRAMS = $(noinst_PROGRAMS)
+am_producer_OBJECTS = example.$(OBJEXT)
+producer_OBJECTS = $(am_producer_OBJECTS)
+am__DEPENDENCIES_1 =
+producer_DEPENDENCIES = $(am__DEPENDENCIES_1)
+am_tests_encoder_OBJECTS = encoder_tests.$(OBJEXT)
+tests_encoder_OBJECTS = $(am_tests_encoder_OBJECTS)
+tests_encoder_DEPENDENCIES = $(am__DEPENDENCIES_1)
+am__dirstamp = $(am__leading_dot)dirstamp
+am_tests_encoder_helper_OBJECTS = encoder_helper_tests.$(OBJEXT)
+tests_encoder_helper_OBJECTS = $(am_tests_encoder_helper_OBJECTS)
+tests_encoder_helper_DEPENDENCIES = $(am__DEPENDENCIES_1)
+am_tests_producer_OBJECTS = producer_tests.$(OBJEXT)
+tests_producer_OBJECTS = $(am_tests_producer_OBJECTS)
+tests_producer_DEPENDENCIES = $(am__DEPENDENCIES_1)
+DEFAULT_INCLUDES = -I.@am__isrc@
+depcomp = $(SHELL) $(top_srcdir)/build-aux/depcomp
+am__depfiles_maybe = depfiles
+am__mv = mv -f
+CXXCOMPILE = $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) \
+ $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS)
+LTCXXCOMPILE = $(LIBTOOL) --tag=CXX $(AM_LIBTOOLFLAGS) $(LIBTOOLFLAGS) \
+ --mode=compile $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) \
+ $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS)
+CXXLD = $(CXX)
+CXXLINK = $(LIBTOOL) --tag=CXX $(AM_LIBTOOLFLAGS) $(LIBTOOLFLAGS) \
+ --mode=link $(CXXLD) $(AM_CXXFLAGS) $(CXXFLAGS) $(AM_LDFLAGS) \
+ $(LDFLAGS) -o $@
+SOURCES = $(libkafkaconnect_la_SOURCES) $(producer_SOURCES) \
+ $(tests_encoder_SOURCES) $(tests_encoder_helper_SOURCES) \
+ $(tests_producer_SOURCES)
+DIST_SOURCES = $(libkafkaconnect_la_SOURCES) $(producer_SOURCES) \
+ $(tests_encoder_SOURCES) $(tests_encoder_helper_SOURCES) \
+ $(tests_producer_SOURCES)
+HEADERS = $(kafkaconnect_include_HEADERS)
+ETAGS = etags
+CTAGS = ctags
+am__tty_colors = \
+red=; grn=; lgn=; blu=; std=
+DISTFILES = $(DIST_COMMON) $(DIST_SOURCES) $(TEXINFOS) $(EXTRA_DIST)
+distdir = $(PACKAGE)-$(VERSION)
+top_distdir = $(distdir)
+am__remove_distdir = \
+ { test ! -d "$(distdir)" \
+ || { find "$(distdir)" -type d ! -perm -200 -exec chmod u+w {} ';' \
+ && rm -fr "$(distdir)"; }; }
+DIST_ARCHIVES = $(distdir).tar.gz
+GZIP_ENV = --best
+distuninstallcheck_listfiles = find . -type f -print
+distcleancheck_listfiles = find . -type f -print
+ACLOCAL = @ACLOCAL@
+AMTAR = @AMTAR@
+AR = @AR@
+AUTOCONF = @AUTOCONF@
+AUTOHEADER = @AUTOHEADER@
+AUTOMAKE = @AUTOMAKE@
+AWK = @AWK@
+CC = @CC@
+CCDEPMODE = @CCDEPMODE@
+CFLAGS = @CFLAGS@
+CPP = @CPP@
+CPPFLAGS = @CPPFLAGS@
+CXX = @CXX@
+CXXCPP = @CXXCPP@
+CXXDEPMODE = @CXXDEPMODE@
+CXXFLAGS = @CXXFLAGS@
+CYGPATH_W = @CYGPATH_W@
+DEFS = @DEFS@
+DEPDIR = @DEPDIR@
+DSYMUTIL = @DSYMUTIL@
+DUMPBIN = @DUMPBIN@
+ECHO_C = @ECHO_C@
+ECHO_N = @ECHO_N@
+ECHO_T = @ECHO_T@
+EGREP = @EGREP@
+EXEEXT = @EXEEXT@
+FGREP = @FGREP@
+GREP = @GREP@
+INSTALL = @INSTALL@
+INSTALL_DATA = @INSTALL_DATA@
+INSTALL_PROGRAM = @INSTALL_PROGRAM@
+INSTALL_SCRIPT = @INSTALL_SCRIPT@
+INSTALL_STRIP_PROGRAM = @INSTALL_STRIP_PROGRAM@
+KAFKACONNECT_VERSION = @KAFKACONNECT_VERSION@
+LD = @LD@
+LDFLAGS = @LDFLAGS@
+LIBOBJS = @LIBOBJS@
+LIBS = @LIBS@
+LIBTOOL = @LIBTOOL@
+LIPO = @LIPO@
+LN_S = @LN_S@
+LTLIBOBJS = @LTLIBOBJS@
+MAKEINFO = @MAKEINFO@
+MKDIR_P = @MKDIR_P@
+NM = @NM@
+NMEDIT = @NMEDIT@
+OBJDUMP = @OBJDUMP@
+OBJEXT = @OBJEXT@
+OTOOL = @OTOOL@
+OTOOL64 = @OTOOL64@
+PACKAGE = @PACKAGE@
+PACKAGE_BUGREPORT = @PACKAGE_BUGREPORT@
+PACKAGE_NAME = @PACKAGE_NAME@
+PACKAGE_STRING = @PACKAGE_STRING@
+PACKAGE_TARNAME = @PACKAGE_TARNAME@
+PACKAGE_URL = @PACKAGE_URL@
+PACKAGE_VERSION = @PACKAGE_VERSION@
+PATH_SEPARATOR = @PATH_SEPARATOR@
+RANLIB = @RANLIB@
+SED = @SED@
+SET_MAKE = @SET_MAKE@
+SHELL = @SHELL@
+STRIP = @STRIP@
+VERSION = @VERSION@
+abs_builddir = @abs_builddir@
+abs_srcdir = @abs_srcdir@
+abs_top_builddir = @abs_top_builddir@
+abs_top_srcdir = @abs_top_srcdir@
+ac_ct_CC = @ac_ct_CC@
+ac_ct_CXX = @ac_ct_CXX@
+ac_ct_DUMPBIN = @ac_ct_DUMPBIN@
+am__include = @am__include@
+am__leading_dot = @am__leading_dot@
+am__quote = @am__quote@
+am__tar = @am__tar@
+am__untar = @am__untar@
+bindir = @bindir@
+build = @build@
+build_alias = @build_alias@
+build_cpu = @build_cpu@
+build_os = @build_os@
+build_vendor = @build_vendor@
+builddir = @builddir@
+datadir = @datadir@
+datarootdir = @datarootdir@
+docdir = @docdir@
+dvidir = @dvidir@
+exec_prefix = @exec_prefix@
+host = @host@
+host_alias = @host_alias@
+host_cpu = @host_cpu@
+host_os = @host_os@
+host_vendor = @host_vendor@
+htmldir = @htmldir@
+includedir = @includedir@
+infodir = @infodir@
+install_sh = @install_sh@
+libdir = @libdir@
+libexecdir = @libexecdir@
+localedir = @localedir@
+localstatedir = @localstatedir@
+lt_ECHO = @lt_ECHO@
+mandir = @mandir@
+mkdir_p = @mkdir_p@
+oldincludedir = @oldincludedir@
+pdfdir = @pdfdir@
+prefix = @prefix@
+program_transform_name = @program_transform_name@
+psdir = @psdir@
+sbindir = @sbindir@
+sharedstatedir = @sharedstatedir@
+srcdir = @srcdir@
+sysconfdir = @sysconfdir@
+target_alias = @target_alias@
+top_build_prefix = @top_build_prefix@
+top_builddir = @top_builddir@
+top_srcdir = @top_srcdir@
+ACLOCAL_AMFLAGS = -I build-aux/m4 ${ACLOCAL_FLAGS}
+AM_CPPFLAGS = $(DEPS_CFLAGS)
+EXAMPLE_LIBS = -lboost_system -lboost_thread -lkafkaconnect
+
+#
+# Shared Library
+#
+lib_LTLIBRARIES = libkafkaconnect.la
+libkafkaconnect_la_SOURCES = src/producer.cpp
+libkafkaconnect_la_LDFLAGS = -version-info $(KAFKACONNECT_VERSION)
+kafkaconnect_includedir = $(includedir)/kafkaconnect
+kafkaconnect_include_HEADERS = src/producer.hpp \
+ src/encoder.hpp \
+ src/encoder_helper.hpp
+
+producer_SOURCES = src/example.cpp
+producer_LDADD = $(DEPS_LIBS) $(EXAMPLE_LIBS)
+tests_encoder_helper_SOURCES = src/tests/encoder_helper_tests.cpp
+tests_encoder_helper_LDADD = $(DEPS_LIBS) $(EXAMPLE_LIBS) -lboost_unit_test_framework
+tests_encoder_SOURCES = src/tests/encoder_tests.cpp
+tests_encoder_LDADD = $(DEPS_LIBS) $(EXAMPLE_LIBS) -lboost_unit_test_framework
+tests_producer_SOURCES = src/tests/producer_tests.cpp
+tests_producer_LDADD = $(DEPS_LIBS) $(EXAMPLE_LIBS) -lboost_unit_test_framework
+all: all-am
+
+.SUFFIXES:
+.SUFFIXES: .cpp .lo .o .obj
+am--refresh:
+ @:
+$(srcdir)/Makefile.in: $(srcdir)/Makefile.am $(am__configure_deps)
+ @for dep in $?; do \
+ case '$(am__configure_deps)' in \
+ *$$dep*) \
+ echo ' cd $(srcdir) && $(AUTOMAKE) --foreign'; \
+ $(am__cd) $(srcdir) && $(AUTOMAKE) --foreign \
+ && exit 0; \
+ exit 1;; \
+ esac; \
+ done; \
+ echo ' cd $(top_srcdir) && $(AUTOMAKE) --foreign Makefile'; \
+ $(am__cd) $(top_srcdir) && \
+ $(AUTOMAKE) --foreign Makefile
+.PRECIOUS: Makefile
+Makefile: $(srcdir)/Makefile.in $(top_builddir)/config.status
+ @case '$?' in \
+ *config.status*) \
+ echo ' $(SHELL) ./config.status'; \
+ $(SHELL) ./config.status;; \
+ *) \
+ echo ' cd $(top_builddir) && $(SHELL) ./config.status $@ $(am__depfiles_maybe)'; \
+ cd $(top_builddir) && $(SHELL) ./config.status $@ $(am__depfiles_maybe);; \
+ esac;
+
+$(top_builddir)/config.status: $(top_srcdir)/configure $(CONFIG_STATUS_DEPENDENCIES)
+ $(SHELL) ./config.status --recheck
+
+$(top_srcdir)/configure: $(am__configure_deps)
+ $(am__cd) $(srcdir) && $(AUTOCONF)
+$(ACLOCAL_M4): $(am__aclocal_m4_deps)
+ $(am__cd) $(srcdir) && $(ACLOCAL) $(ACLOCAL_AMFLAGS)
+$(am__aclocal_m4_deps):
+install-libLTLIBRARIES: $(lib_LTLIBRARIES)
+ @$(NORMAL_INSTALL)
+ test -z "$(libdir)" || $(MKDIR_P) "$(DESTDIR)$(libdir)"
+ @list='$(lib_LTLIBRARIES)'; test -n "$(libdir)" || list=; \
+ list2=; for p in $$list; do \
+ if test -f $$p; then \
+ list2="$$list2 $$p"; \
+ else :; fi; \
+ done; \
+ test -z "$$list2" || { \
+ echo " $(LIBTOOL) $(AM_LIBTOOLFLAGS) $(LIBTOOLFLAGS) --mode=install $(INSTALL) $(INSTALL_STRIP_FLAG) $$list2 '$(DESTDIR)$(libdir)'"; \
+ $(LIBTOOL) $(AM_LIBTOOLFLAGS) $(LIBTOOLFLAGS) --mode=install $(INSTALL) $(INSTALL_STRIP_FLAG) $$list2 "$(DESTDIR)$(libdir)"; \
+ }
+
+uninstall-libLTLIBRARIES:
+ @$(NORMAL_UNINSTALL)
+ @list='$(lib_LTLIBRARIES)'; test -n "$(libdir)" || list=; \
+ for p in $$list; do \
+ $(am__strip_dir) \
+ echo " $(LIBTOOL) $(AM_LIBTOOLFLAGS) $(LIBTOOLFLAGS) --mode=uninstall rm -f '$(DESTDIR)$(libdir)/$$f'"; \
+ $(LIBTOOL) $(AM_LIBTOOLFLAGS) $(LIBTOOLFLAGS) --mode=uninstall rm -f "$(DESTDIR)$(libdir)/$$f"; \
+ done
+
+clean-libLTLIBRARIES:
+ -test -z "$(lib_LTLIBRARIES)" || rm -f $(lib_LTLIBRARIES)
+ @list='$(lib_LTLIBRARIES)'; for p in $$list; do \
+ dir="`echo $$p | sed -e 's|/[^/]*$$||'`"; \
+ test "$$dir" != "$$p" || dir=.; \
+ echo "rm -f \"$${dir}/so_locations\""; \
+ rm -f "$${dir}/so_locations"; \
+ done
+libkafkaconnect.la: $(libkafkaconnect_la_OBJECTS) $(libkafkaconnect_la_DEPENDENCIES)
+ $(libkafkaconnect_la_LINK) -rpath $(libdir) $(libkafkaconnect_la_OBJECTS) $(libkafkaconnect_la_LIBADD) $(LIBS)
+
+clean-checkPROGRAMS:
+ @list='$(check_PROGRAMS)'; test -n "$$list" || exit 0; \
+ echo " rm -f" $$list; \
+ rm -f $$list || exit $$?; \
+ test -n "$(EXEEXT)" || exit 0; \
+ list=`for p in $$list; do echo "$$p"; done | sed 's/$(EXEEXT)$$//'`; \
+ echo " rm -f" $$list; \
+ rm -f $$list
+
+clean-noinstPROGRAMS:
+ @list='$(noinst_PROGRAMS)'; test -n "$$list" || exit 0; \
+ echo " rm -f" $$list; \
+ rm -f $$list || exit $$?; \
+ test -n "$(EXEEXT)" || exit 0; \
+ list=`for p in $$list; do echo "$$p"; done | sed 's/$(EXEEXT)$$//'`; \
+ echo " rm -f" $$list; \
+ rm -f $$list
+producer$(EXEEXT): $(producer_OBJECTS) $(producer_DEPENDENCIES)
+ @rm -f producer$(EXEEXT)
+ $(CXXLINK) $(producer_OBJECTS) $(producer_LDADD) $(LIBS)
+tests/$(am__dirstamp):
+ @$(MKDIR_P) tests
+ @: > tests/$(am__dirstamp)
+tests/encoder$(EXEEXT): $(tests_encoder_OBJECTS) $(tests_encoder_DEPENDENCIES) tests/$(am__dirstamp)
+ @rm -f tests/encoder$(EXEEXT)
+ $(CXXLINK) $(tests_encoder_OBJECTS) $(tests_encoder_LDADD) $(LIBS)
+tests/encoder_helper$(EXEEXT): $(tests_encoder_helper_OBJECTS) $(tests_encoder_helper_DEPENDENCIES) tests/$(am__dirstamp)
+ @rm -f tests/encoder_helper$(EXEEXT)
+ $(CXXLINK) $(tests_encoder_helper_OBJECTS) $(tests_encoder_helper_LDADD) $(LIBS)
+tests/producer$(EXEEXT): $(tests_producer_OBJECTS) $(tests_producer_DEPENDENCIES) tests/$(am__dirstamp)
+ @rm -f tests/producer$(EXEEXT)
+ $(CXXLINK) $(tests_producer_OBJECTS) $(tests_producer_LDADD) $(LIBS)
+
+mostlyclean-compile:
+ -rm -f *.$(OBJEXT)
+
+distclean-compile:
+ -rm -f *.tab.c
+
+@AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/encoder_helper_tests.Po@am__quote@
+@AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/encoder_tests.Po@am__quote@
+@AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/example.Po@am__quote@
+@AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/producer.Plo@am__quote@
+@AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/producer_tests.Po@am__quote@
+
+.cpp.o:
+@am__fastdepCXX_TRUE@ $(CXXCOMPILE) -MT $@ -MD -MP -MF $(DEPDIR)/$*.Tpo -c -o $@ $<
+@am__fastdepCXX_TRUE@ $(am__mv) $(DEPDIR)/$*.Tpo $(DEPDIR)/$*.Po
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ source='$<' object='$@' libtool=no @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ DEPDIR=$(DEPDIR) $(CXXDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCXX_FALSE@ $(CXXCOMPILE) -c -o $@ $<
+
+.cpp.obj:
+@am__fastdepCXX_TRUE@ $(CXXCOMPILE) -MT $@ -MD -MP -MF $(DEPDIR)/$*.Tpo -c -o $@ `$(CYGPATH_W) '$<'`
+@am__fastdepCXX_TRUE@ $(am__mv) $(DEPDIR)/$*.Tpo $(DEPDIR)/$*.Po
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ source='$<' object='$@' libtool=no @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ DEPDIR=$(DEPDIR) $(CXXDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCXX_FALSE@ $(CXXCOMPILE) -c -o $@ `$(CYGPATH_W) '$<'`
+
+.cpp.lo:
+@am__fastdepCXX_TRUE@ $(LTCXXCOMPILE) -MT $@ -MD -MP -MF $(DEPDIR)/$*.Tpo -c -o $@ $<
+@am__fastdepCXX_TRUE@ $(am__mv) $(DEPDIR)/$*.Tpo $(DEPDIR)/$*.Plo
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ source='$<' object='$@' libtool=yes @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ DEPDIR=$(DEPDIR) $(CXXDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCXX_FALSE@ $(LTCXXCOMPILE) -c -o $@ $<
+
+producer.lo: src/producer.cpp
+@am__fastdepCXX_TRUE@ $(LIBTOOL) --tag=CXX $(AM_LIBTOOLFLAGS) $(LIBTOOLFLAGS) --mode=compile $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -MT producer.lo -MD -MP -MF $(DEPDIR)/producer.Tpo -c -o producer.lo `test -f 'src/producer.cpp' || echo '$(srcdir)/'`src/producer.cpp
+@am__fastdepCXX_TRUE@ $(am__mv) $(DEPDIR)/producer.Tpo $(DEPDIR)/producer.Plo
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ source='src/producer.cpp' object='producer.lo' libtool=yes @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ DEPDIR=$(DEPDIR) $(CXXDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCXX_FALSE@ $(LIBTOOL) --tag=CXX $(AM_LIBTOOLFLAGS) $(LIBTOOLFLAGS) --mode=compile $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -c -o producer.lo `test -f 'src/producer.cpp' || echo '$(srcdir)/'`src/producer.cpp
+
+example.o: src/example.cpp
+@am__fastdepCXX_TRUE@ $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -MT example.o -MD -MP -MF $(DEPDIR)/example.Tpo -c -o example.o `test -f 'src/example.cpp' || echo '$(srcdir)/'`src/example.cpp
+@am__fastdepCXX_TRUE@ $(am__mv) $(DEPDIR)/example.Tpo $(DEPDIR)/example.Po
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ source='src/example.cpp' object='example.o' libtool=no @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ DEPDIR=$(DEPDIR) $(CXXDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCXX_FALSE@ $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -c -o example.o `test -f 'src/example.cpp' || echo '$(srcdir)/'`src/example.cpp
+
+example.obj: src/example.cpp
+@am__fastdepCXX_TRUE@ $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -MT example.obj -MD -MP -MF $(DEPDIR)/example.Tpo -c -o example.obj `if test -f 'src/example.cpp'; then $(CYGPATH_W) 'src/example.cpp'; else $(CYGPATH_W) '$(srcdir)/src/example.cpp'; fi`
+@am__fastdepCXX_TRUE@ $(am__mv) $(DEPDIR)/example.Tpo $(DEPDIR)/example.Po
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ source='src/example.cpp' object='example.obj' libtool=no @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ DEPDIR=$(DEPDIR) $(CXXDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCXX_FALSE@ $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -c -o example.obj `if test -f 'src/example.cpp'; then $(CYGPATH_W) 'src/example.cpp'; else $(CYGPATH_W) '$(srcdir)/src/example.cpp'; fi`
+
+encoder_tests.o: src/tests/encoder_tests.cpp
+@am__fastdepCXX_TRUE@ $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -MT encoder_tests.o -MD -MP -MF $(DEPDIR)/encoder_tests.Tpo -c -o encoder_tests.o `test -f 'src/tests/encoder_tests.cpp' || echo '$(srcdir)/'`src/tests/encoder_tests.cpp
+@am__fastdepCXX_TRUE@ $(am__mv) $(DEPDIR)/encoder_tests.Tpo $(DEPDIR)/encoder_tests.Po
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ source='src/tests/encoder_tests.cpp' object='encoder_tests.o' libtool=no @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ DEPDIR=$(DEPDIR) $(CXXDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCXX_FALSE@ $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -c -o encoder_tests.o `test -f 'src/tests/encoder_tests.cpp' || echo '$(srcdir)/'`src/tests/encoder_tests.cpp
+
+encoder_tests.obj: src/tests/encoder_tests.cpp
+@am__fastdepCXX_TRUE@ $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -MT encoder_tests.obj -MD -MP -MF $(DEPDIR)/encoder_tests.Tpo -c -o encoder_tests.obj `if test -f 'src/tests/encoder_tests.cpp'; then $(CYGPATH_W) 'src/tests/encoder_tests.cpp'; else $(CYGPATH_W) '$(srcdir)/src/tests/encoder_tests.cpp'; fi`
+@am__fastdepCXX_TRUE@ $(am__mv) $(DEPDIR)/encoder_tests.Tpo $(DEPDIR)/encoder_tests.Po
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ source='src/tests/encoder_tests.cpp' object='encoder_tests.obj' libtool=no @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ DEPDIR=$(DEPDIR) $(CXXDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCXX_FALSE@ $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -c -o encoder_tests.obj `if test -f 'src/tests/encoder_tests.cpp'; then $(CYGPATH_W) 'src/tests/encoder_tests.cpp'; else $(CYGPATH_W) '$(srcdir)/src/tests/encoder_tests.cpp'; fi`
+
+encoder_helper_tests.o: src/tests/encoder_helper_tests.cpp
+@am__fastdepCXX_TRUE@ $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -MT encoder_helper_tests.o -MD -MP -MF $(DEPDIR)/encoder_helper_tests.Tpo -c -o encoder_helper_tests.o `test -f 'src/tests/encoder_helper_tests.cpp' || echo '$(srcdir)/'`src/tests/encoder_helper_tests.cpp
+@am__fastdepCXX_TRUE@ $(am__mv) $(DEPDIR)/encoder_helper_tests.Tpo $(DEPDIR)/encoder_helper_tests.Po
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ source='src/tests/encoder_helper_tests.cpp' object='encoder_helper_tests.o' libtool=no @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ DEPDIR=$(DEPDIR) $(CXXDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCXX_FALSE@ $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -c -o encoder_helper_tests.o `test -f 'src/tests/encoder_helper_tests.cpp' || echo '$(srcdir)/'`src/tests/encoder_helper_tests.cpp
+
+encoder_helper_tests.obj: src/tests/encoder_helper_tests.cpp
+@am__fastdepCXX_TRUE@ $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -MT encoder_helper_tests.obj -MD -MP -MF $(DEPDIR)/encoder_helper_tests.Tpo -c -o encoder_helper_tests.obj `if test -f 'src/tests/encoder_helper_tests.cpp'; then $(CYGPATH_W) 'src/tests/encoder_helper_tests.cpp'; else $(CYGPATH_W) '$(srcdir)/src/tests/encoder_helper_tests.cpp'; fi`
+@am__fastdepCXX_TRUE@ $(am__mv) $(DEPDIR)/encoder_helper_tests.Tpo $(DEPDIR)/encoder_helper_tests.Po
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ source='src/tests/encoder_helper_tests.cpp' object='encoder_helper_tests.obj' libtool=no @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ DEPDIR=$(DEPDIR) $(CXXDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCXX_FALSE@ $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -c -o encoder_helper_tests.obj `if test -f 'src/tests/encoder_helper_tests.cpp'; then $(CYGPATH_W) 'src/tests/encoder_helper_tests.cpp'; else $(CYGPATH_W) '$(srcdir)/src/tests/encoder_helper_tests.cpp'; fi`
+
+producer_tests.o: src/tests/producer_tests.cpp
+@am__fastdepCXX_TRUE@ $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -MT producer_tests.o -MD -MP -MF $(DEPDIR)/producer_tests.Tpo -c -o producer_tests.o `test -f 'src/tests/producer_tests.cpp' || echo '$(srcdir)/'`src/tests/producer_tests.cpp
+@am__fastdepCXX_TRUE@ $(am__mv) $(DEPDIR)/producer_tests.Tpo $(DEPDIR)/producer_tests.Po
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ source='src/tests/producer_tests.cpp' object='producer_tests.o' libtool=no @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ DEPDIR=$(DEPDIR) $(CXXDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCXX_FALSE@ $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -c -o producer_tests.o `test -f 'src/tests/producer_tests.cpp' || echo '$(srcdir)/'`src/tests/producer_tests.cpp
+
+producer_tests.obj: src/tests/producer_tests.cpp
+@am__fastdepCXX_TRUE@ $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -MT producer_tests.obj -MD -MP -MF $(DEPDIR)/producer_tests.Tpo -c -o producer_tests.obj `if test -f 'src/tests/producer_tests.cpp'; then $(CYGPATH_W) 'src/tests/producer_tests.cpp'; else $(CYGPATH_W) '$(srcdir)/src/tests/producer_tests.cpp'; fi`
+@am__fastdepCXX_TRUE@ $(am__mv) $(DEPDIR)/producer_tests.Tpo $(DEPDIR)/producer_tests.Po
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ source='src/tests/producer_tests.cpp' object='producer_tests.obj' libtool=no @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCXX_FALSE@ DEPDIR=$(DEPDIR) $(CXXDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCXX_FALSE@ $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CXXFLAGS) $(CXXFLAGS) -c -o producer_tests.obj `if test -f 'src/tests/producer_tests.cpp'; then $(CYGPATH_W) 'src/tests/producer_tests.cpp'; else $(CYGPATH_W) '$(srcdir)/src/tests/producer_tests.cpp'; fi`
+
+mostlyclean-libtool:
+ -rm -f *.lo
+
+clean-libtool:
+ -rm -rf .libs _libs
+ -rm -rf tests/.libs tests/_libs
+
+distclean-libtool:
+ -rm -f libtool config.lt
+install-kafkaconnect_includeHEADERS: $(kafkaconnect_include_HEADERS)
+ @$(NORMAL_INSTALL)
+ test -z "$(kafkaconnect_includedir)" || $(MKDIR_P) "$(DESTDIR)$(kafkaconnect_includedir)"
+ @list='$(kafkaconnect_include_HEADERS)'; test -n "$(kafkaconnect_includedir)" || list=; \
+ for p in $$list; do \
+ if test -f "$$p"; then d=; else d="$(srcdir)/"; fi; \
+ echo "$$d$$p"; \
+ done | $(am__base_list) | \
+ while read files; do \
+ echo " $(INSTALL_HEADER) $$files '$(DESTDIR)$(kafkaconnect_includedir)'"; \
+ $(INSTALL_HEADER) $$files "$(DESTDIR)$(kafkaconnect_includedir)" || exit $$?; \
+ done
+
+uninstall-kafkaconnect_includeHEADERS:
+ @$(NORMAL_UNINSTALL)
+ @list='$(kafkaconnect_include_HEADERS)'; test -n "$(kafkaconnect_includedir)" || list=; \
+ files=`for p in $$list; do echo $$p; done | sed -e 's|^.*/||'`; \
+ test -n "$$files" || exit 0; \
+ echo " ( cd '$(DESTDIR)$(kafkaconnect_includedir)' && rm -f" $$files ")"; \
+ cd "$(DESTDIR)$(kafkaconnect_includedir)" && rm -f $$files
+
+ID: $(HEADERS) $(SOURCES) $(LISP) $(TAGS_FILES)
+ list='$(SOURCES) $(HEADERS) $(LISP) $(TAGS_FILES)'; \
+ unique=`for i in $$list; do \
+ if test -f "$$i"; then echo $$i; else echo $(srcdir)/$$i; fi; \
+ done | \
+ $(AWK) '{ files[$$0] = 1; nonempty = 1; } \
+ END { if (nonempty) { for (i in files) print i; }; }'`; \
+ mkid -fID $$unique
+tags: TAGS
+
+TAGS: $(HEADERS) $(SOURCES) $(TAGS_DEPENDENCIES) \
+ $(TAGS_FILES) $(LISP)
+ set x; \
+ here=`pwd`; \
+ list='$(SOURCES) $(HEADERS) $(LISP) $(TAGS_FILES)'; \
+ unique=`for i in $$list; do \
+ if test -f "$$i"; then echo $$i; else echo $(srcdir)/$$i; fi; \
+ done | \
+ $(AWK) '{ files[$$0] = 1; nonempty = 1; } \
+ END { if (nonempty) { for (i in files) print i; }; }'`; \
+ shift; \
+ if test -z "$(ETAGS_ARGS)$$*$$unique"; then :; else \
+ test -n "$$unique" || unique=$$empty_fix; \
+ if test $$# -gt 0; then \
+ $(ETAGS) $(ETAGSFLAGS) $(AM_ETAGSFLAGS) $(ETAGS_ARGS) \
+ "$$@" $$unique; \
+ else \
+ $(ETAGS) $(ETAGSFLAGS) $(AM_ETAGSFLAGS) $(ETAGS_ARGS) \
+ $$unique; \
+ fi; \
+ fi
+ctags: CTAGS
+CTAGS: $(HEADERS) $(SOURCES) $(TAGS_DEPENDENCIES) \
+ $(TAGS_FILES) $(LISP)
+ list='$(SOURCES) $(HEADERS) $(LISP) $(TAGS_FILES)'; \
+ unique=`for i in $$list; do \
+ if test -f "$$i"; then echo $$i; else echo $(srcdir)/$$i; fi; \
+ done | \
+ $(AWK) '{ files[$$0] = 1; nonempty = 1; } \
+ END { if (nonempty) { for (i in files) print i; }; }'`; \
+ test -z "$(CTAGS_ARGS)$$unique" \
+ || $(CTAGS) $(CTAGSFLAGS) $(AM_CTAGSFLAGS) $(CTAGS_ARGS) \
+ $$unique
+
+GTAGS:
+ here=`$(am__cd) $(top_builddir) && pwd` \
+ && $(am__cd) $(top_srcdir) \
+ && gtags -i $(GTAGS_ARGS) "$$here"
+
+distclean-tags:
+ -rm -f TAGS ID GTAGS GRTAGS GSYMS GPATH tags
+
+check-TESTS: $(TESTS)
+ @failed=0; all=0; xfail=0; xpass=0; skip=0; \
+ srcdir=$(srcdir); export srcdir; \
+ list=' $(TESTS) '; \
+ $(am__tty_colors); \
+ if test -n "$$list"; then \
+ for tst in $$list; do \
+ if test -f ./$$tst; then dir=./; \
+ elif test -f $$tst; then dir=; \
+ else dir="$(srcdir)/"; fi; \
+ if $(TESTS_ENVIRONMENT) $${dir}$$tst; then \
+ all=`expr $$all + 1`; \
+ case " $(XFAIL_TESTS) " in \
+ *[\ \ ]$$tst[\ \ ]*) \
+ xpass=`expr $$xpass + 1`; \
+ failed=`expr $$failed + 1`; \
+ col=$$red; res=XPASS; \
+ ;; \
+ *) \
+ col=$$grn; res=PASS; \
+ ;; \
+ esac; \
+ elif test $$? -ne 77; then \
+ all=`expr $$all + 1`; \
+ case " $(XFAIL_TESTS) " in \
+ *[\ \ ]$$tst[\ \ ]*) \
+ xfail=`expr $$xfail + 1`; \
+ col=$$lgn; res=XFAIL; \
+ ;; \
+ *) \
+ failed=`expr $$failed + 1`; \
+ col=$$red; res=FAIL; \
+ ;; \
+ esac; \
+ else \
+ skip=`expr $$skip + 1`; \
+ col=$$blu; res=SKIP; \
+ fi; \
+ echo "$${col}$$res$${std}: $$tst"; \
+ done; \
+ if test "$$all" -eq 1; then \
+ tests="test"; \
+ All=""; \
+ else \
+ tests="tests"; \
+ All="All "; \
+ fi; \
+ if test "$$failed" -eq 0; then \
+ if test "$$xfail" -eq 0; then \
+ banner="$$All$$all $$tests passed"; \
+ else \
+ if test "$$xfail" -eq 1; then failures=failure; else failures=failures; fi; \
+ banner="$$All$$all $$tests behaved as expected ($$xfail expected $$failures)"; \
+ fi; \
+ else \
+ if test "$$xpass" -eq 0; then \
+ banner="$$failed of $$all $$tests failed"; \
+ else \
+ if test "$$xpass" -eq 1; then passes=pass; else passes=passes; fi; \
+ banner="$$failed of $$all $$tests did not behave as expected ($$xpass unexpected $$passes)"; \
+ fi; \
+ fi; \
+ dashes="$$banner"; \
+ skipped=""; \
+ if test "$$skip" -ne 0; then \
+ if test "$$skip" -eq 1; then \
+ skipped="($$skip test was not run)"; \
+ else \
+ skipped="($$skip tests were not run)"; \
+ fi; \
+ test `echo "$$skipped" | wc -c` -le `echo "$$banner" | wc -c` || \
+ dashes="$$skipped"; \
+ fi; \
+ report=""; \
+ if test "$$failed" -ne 0 && test -n "$(PACKAGE_BUGREPORT)"; then \
+ report="Please report to $(PACKAGE_BUGREPORT)"; \
+ test `echo "$$report" | wc -c` -le `echo "$$banner" | wc -c` || \
+ dashes="$$report"; \
+ fi; \
+ dashes=`echo "$$dashes" | sed s/./=/g`; \
+ if test "$$failed" -eq 0; then \
+ echo "$$grn$$dashes"; \
+ else \
+ echo "$$red$$dashes"; \
+ fi; \
+ echo "$$banner"; \
+ test -z "$$skipped" || echo "$$skipped"; \
+ test -z "$$report" || echo "$$report"; \
+ echo "$$dashes$$std"; \
+ test "$$failed" -eq 0; \
+ else :; fi
+
+distdir: $(DISTFILES)
+ $(am__remove_distdir)
+ test -d "$(distdir)" || mkdir "$(distdir)"
+ @srcdirstrip=`echo "$(srcdir)" | sed 's/[].[^$$\\*]/\\\\&/g'`; \
+ topsrcdirstrip=`echo "$(top_srcdir)" | sed 's/[].[^$$\\*]/\\\\&/g'`; \
+ list='$(DISTFILES)'; \
+ dist_files=`for file in $$list; do echo $$file; done | \
+ sed -e "s|^$$srcdirstrip/||;t" \
+ -e "s|^$$topsrcdirstrip/|$(top_builddir)/|;t"`; \
+ case $$dist_files in \
+ */*) $(MKDIR_P) `echo "$$dist_files" | \
+ sed '/\//!d;s|^|$(distdir)/|;s,/[^/]*$$,,' | \
+ sort -u` ;; \
+ esac; \
+ for file in $$dist_files; do \
+ if test -f $$file || test -d $$file; then d=.; else d=$(srcdir); fi; \
+ if test -d $$d/$$file; then \
+ dir=`echo "/$$file" | sed -e 's,/[^/]*$$,,'`; \
+ if test -d "$(distdir)/$$file"; then \
+ find "$(distdir)/$$file" -type d ! -perm -700 -exec chmod u+rwx {} \;; \
+ fi; \
+ if test -d $(srcdir)/$$file && test $$d != $(srcdir); then \
+ cp -fpR $(srcdir)/$$file "$(distdir)$$dir" || exit 1; \
+ find "$(distdir)/$$file" -type d ! -perm -700 -exec chmod u+rwx {} \;; \
+ fi; \
+ cp -fpR $$d/$$file "$(distdir)$$dir" || exit 1; \
+ else \
+ test -f "$(distdir)/$$file" \
+ || cp -p $$d/$$file "$(distdir)/$$file" \
+ || exit 1; \
+ fi; \
+ done
+ -test -n "$(am__skip_mode_fix)" \
+ || find "$(distdir)" -type d ! -perm -755 \
+ -exec chmod u+rwx,go+rx {} \; -o \
+ ! -type d ! -perm -444 -links 1 -exec chmod a+r {} \; -o \
+ ! -type d ! -perm -400 -exec chmod a+r {} \; -o \
+ ! -type d ! -perm -444 -exec $(install_sh) -c -m a+r {} {} \; \
+ || chmod -R a+r "$(distdir)"
+dist-gzip: distdir
+ tardir=$(distdir) && $(am__tar) | GZIP=$(GZIP_ENV) gzip -c >$(distdir).tar.gz
+ $(am__remove_distdir)
+
+dist-bzip2: distdir
+ tardir=$(distdir) && $(am__tar) | bzip2 -9 -c >$(distdir).tar.bz2
+ $(am__remove_distdir)
+
+dist-lzma: distdir
+ tardir=$(distdir) && $(am__tar) | lzma -9 -c >$(distdir).tar.lzma
+ $(am__remove_distdir)
+
+dist-xz: distdir
+ tardir=$(distdir) && $(am__tar) | xz -c >$(distdir).tar.xz
+ $(am__remove_distdir)
+
+dist-tarZ: distdir
+ tardir=$(distdir) && $(am__tar) | compress -c >$(distdir).tar.Z
+ $(am__remove_distdir)
+
+dist-shar: distdir
+ shar $(distdir) | GZIP=$(GZIP_ENV) gzip -c >$(distdir).shar.gz
+ $(am__remove_distdir)
+
+dist-zip: distdir
+ -rm -f $(distdir).zip
+ zip -rq $(distdir).zip $(distdir)
+ $(am__remove_distdir)
+
+dist dist-all: distdir
+ tardir=$(distdir) && $(am__tar) | GZIP=$(GZIP_ENV) gzip -c >$(distdir).tar.gz
+ $(am__remove_distdir)
+
+# This target untars the dist file and tries a VPATH configuration. Then
+# it guarantees that the distribution is self-contained by making another
+# tarfile.
+distcheck: dist
+ case '$(DIST_ARCHIVES)' in \
+ *.tar.gz*) \
+ GZIP=$(GZIP_ENV) gzip -dc $(distdir).tar.gz | $(am__untar) ;;\
+ *.tar.bz2*) \
+ bzip2 -dc $(distdir).tar.bz2 | $(am__untar) ;;\
+ *.tar.lzma*) \
+ lzma -dc $(distdir).tar.lzma | $(am__untar) ;;\
+ *.tar.xz*) \
+ xz -dc $(distdir).tar.xz | $(am__untar) ;;\
+ *.tar.Z*) \
+ uncompress -c $(distdir).tar.Z | $(am__untar) ;;\
+ *.shar.gz*) \
+ GZIP=$(GZIP_ENV) gzip -dc $(distdir).shar.gz | unshar ;;\
+ *.zip*) \
+ unzip $(distdir).zip ;;\
+ esac
+ chmod -R a-w $(distdir); chmod a+w $(distdir)
+ mkdir $(distdir)/_build
+ mkdir $(distdir)/_inst
+ chmod a-w $(distdir)
+ test -d $(distdir)/_build || exit 0; \
+ dc_install_base=`$(am__cd) $(distdir)/_inst && pwd | sed -e 's,^[^:\\/]:[\\/],/,'` \
+ && dc_destdir="$${TMPDIR-/tmp}/am-dc-$$$$/" \
+ && am__cwd=`pwd` \
+ && $(am__cd) $(distdir)/_build \
+ && ../configure --srcdir=.. --prefix="$$dc_install_base" \
+ $(DISTCHECK_CONFIGURE_FLAGS) \
+ && $(MAKE) $(AM_MAKEFLAGS) \
+ && $(MAKE) $(AM_MAKEFLAGS) dvi \
+ && $(MAKE) $(AM_MAKEFLAGS) check \
+ && $(MAKE) $(AM_MAKEFLAGS) install \
+ && $(MAKE) $(AM_MAKEFLAGS) installcheck \
+ && $(MAKE) $(AM_MAKEFLAGS) uninstall \
+ && $(MAKE) $(AM_MAKEFLAGS) distuninstallcheck_dir="$$dc_install_base" \
+ distuninstallcheck \
+ && chmod -R a-w "$$dc_install_base" \
+ && ({ \
+ (cd ../.. && umask 077 && mkdir "$$dc_destdir") \
+ && $(MAKE) $(AM_MAKEFLAGS) DESTDIR="$$dc_destdir" install \
+ && $(MAKE) $(AM_MAKEFLAGS) DESTDIR="$$dc_destdir" uninstall \
+ && $(MAKE) $(AM_MAKEFLAGS) DESTDIR="$$dc_destdir" \
+ distuninstallcheck_dir="$$dc_destdir" distuninstallcheck; \
+ } || { rm -rf "$$dc_destdir"; exit 1; }) \
+ && rm -rf "$$dc_destdir" \
+ && $(MAKE) $(AM_MAKEFLAGS) dist \
+ && rm -rf $(DIST_ARCHIVES) \
+ && $(MAKE) $(AM_MAKEFLAGS) distcleancheck \
+ && cd "$$am__cwd" \
+ || exit 1
+ $(am__remove_distdir)
+ @(echo "$(distdir) archives ready for distribution: "; \
+ list='$(DIST_ARCHIVES)'; for i in $$list; do echo $$i; done) | \
+ sed -e 1h -e 1s/./=/g -e 1p -e 1x -e '$$p' -e '$$x'
+distuninstallcheck:
+ @$(am__cd) '$(distuninstallcheck_dir)' \
+ && test `$(distuninstallcheck_listfiles) | wc -l` -le 1 \
+ || { echo "ERROR: files left after uninstall:" ; \
+ if test -n "$(DESTDIR)"; then \
+ echo " (check DESTDIR support)"; \
+ fi ; \
+ $(distuninstallcheck_listfiles) ; \
+ exit 1; } >&2
+distcleancheck: distclean
+ @if test '$(srcdir)' = . ; then \
+ echo "ERROR: distcleancheck can only run from a VPATH build" ; \
+ exit 1 ; \
+ fi
+ @test `$(distcleancheck_listfiles) | wc -l` -eq 0 \
+ || { echo "ERROR: files left in build directory after distclean:" ; \
+ $(distcleancheck_listfiles) ; \
+ exit 1; } >&2
+check-am: all-am
+ $(MAKE) $(AM_MAKEFLAGS) $(check_PROGRAMS)
+ $(MAKE) $(AM_MAKEFLAGS) check-TESTS
+check: check-am
+all-am: Makefile $(LTLIBRARIES) $(PROGRAMS) $(HEADERS)
+installdirs:
+ for dir in "$(DESTDIR)$(libdir)" "$(DESTDIR)$(kafkaconnect_includedir)"; do \
+ test -z "$$dir" || $(MKDIR_P) "$$dir"; \
+ done
+install: install-am
+install-exec: install-exec-am
+install-data: install-data-am
+uninstall: uninstall-am
+
+install-am: all-am
+ @$(MAKE) $(AM_MAKEFLAGS) install-exec-am install-data-am
+
+installcheck: installcheck-am
+install-strip:
+ $(MAKE) $(AM_MAKEFLAGS) INSTALL_PROGRAM="$(INSTALL_STRIP_PROGRAM)" \
+ install_sh_PROGRAM="$(INSTALL_STRIP_PROGRAM)" INSTALL_STRIP_FLAG=-s \
+ `test -z '$(STRIP)' || \
+ echo "INSTALL_PROGRAM_ENV=STRIPPROG='$(STRIP)'"` install
+mostlyclean-generic:
+
+clean-generic:
+
+distclean-generic:
+ -test -z "$(CONFIG_CLEAN_FILES)" || rm -f $(CONFIG_CLEAN_FILES)
+ -test . = "$(srcdir)" || test -z "$(CONFIG_CLEAN_VPATH_FILES)" || rm -f $(CONFIG_CLEAN_VPATH_FILES)
+ -rm -f tests/$(am__dirstamp)
+
+maintainer-clean-generic:
+ @echo "This command is intended for maintainers to use"
+ @echo "it deletes files that may require special tools to rebuild."
+clean: clean-am
+
+clean-am: clean-checkPROGRAMS clean-generic clean-libLTLIBRARIES \
+ clean-libtool clean-noinstPROGRAMS mostlyclean-am
+
+distclean: distclean-am
+ -rm -f $(am__CONFIG_DISTCLEAN_FILES)
+ -rm -rf ./$(DEPDIR)
+ -rm -f Makefile
+distclean-am: clean-am distclean-compile distclean-generic \
+ distclean-libtool distclean-tags
+
+dvi: dvi-am
+
+dvi-am:
+
+html: html-am
+
+html-am:
+
+info: info-am
+
+info-am:
+
+install-data-am: install-kafkaconnect_includeHEADERS
+
+install-dvi: install-dvi-am
+
+install-dvi-am:
+
+install-exec-am: install-libLTLIBRARIES
+
+install-html: install-html-am
+
+install-html-am:
+
+install-info: install-info-am
+
+install-info-am:
+
+install-man:
+
+install-pdf: install-pdf-am
+
+install-pdf-am:
+
+install-ps: install-ps-am
+
+install-ps-am:
+
+installcheck-am:
+
+maintainer-clean: maintainer-clean-am
+ -rm -f $(am__CONFIG_DISTCLEAN_FILES)
+ -rm -rf $(top_srcdir)/autom4te.cache
+ -rm -rf ./$(DEPDIR)
+ -rm -f Makefile
+maintainer-clean-am: distclean-am maintainer-clean-generic
+
+mostlyclean: mostlyclean-am
+
+mostlyclean-am: mostlyclean-compile mostlyclean-generic \
+ mostlyclean-libtool
+
+pdf: pdf-am
+
+pdf-am:
+
+ps: ps-am
+
+ps-am:
+
+uninstall-am: uninstall-kafkaconnect_includeHEADERS \
+ uninstall-libLTLIBRARIES
+
+.MAKE: check-am install-am install-strip
+
+.PHONY: CTAGS GTAGS all all-am am--refresh check check-TESTS check-am \
+ clean clean-checkPROGRAMS clean-generic clean-libLTLIBRARIES \
+ clean-libtool clean-noinstPROGRAMS ctags dist dist-all \
+ dist-bzip2 dist-gzip dist-lzma dist-shar dist-tarZ dist-xz \
+ dist-zip distcheck distclean distclean-compile \
+ distclean-generic distclean-libtool distclean-tags \
+ distcleancheck distdir distuninstallcheck dvi dvi-am html \
+ html-am info info-am install install-am install-data \
+ install-data-am install-dvi install-dvi-am install-exec \
+ install-exec-am install-html install-html-am install-info \
+ install-info-am install-kafkaconnect_includeHEADERS \
+ install-libLTLIBRARIES install-man install-pdf install-pdf-am \
+ install-ps install-ps-am install-strip installcheck \
+ installcheck-am installdirs maintainer-clean \
+ maintainer-clean-generic mostlyclean mostlyclean-compile \
+ mostlyclean-generic mostlyclean-libtool pdf pdf-am ps ps-am \
+ tags uninstall uninstall-am \
+ uninstall-kafkaconnect_includeHEADERS uninstall-libLTLIBRARIES
+
+
+# Tell versions [3.59,3.63) of GNU make to not export all variables.
+# Otherwise a system limit (for SysV at least) may be exceeded.
+.NOEXPORT:
Added: incubator/kafka/trunk/clients/cpp/README.md
URL: http://svn.apache.org/viewvc/incubator/kafka/trunk/clients/cpp/README.md?rev=1152970&view=auto
==============================================================================
--- incubator/kafka/trunk/clients/cpp/README.md (added)
+++ incubator/kafka/trunk/clients/cpp/README.md Mon Aug 1 23:41:24 2011
@@ -0,0 +1,55 @@
+# C++ kafka library
+This library allows you to produce messages to the Kafka distributed publish/subscribe messaging service.
+
+## Requirements
+Tested on Ubuntu and Redhat both with g++ 4.4 and Boost 1.46.1
+
+## Installation
+Make sure you have g++ and the latest version of Boost:
+http://gcc.gnu.org/
+http://www.boost.org/
+
+```bash
+./configure
+```
+
+Run this to generate the makefile for your system. Do this first.
+
+
+```bash
+make
+```
+
+builds the producer example and the KafkaConnect library
+
+
+```bash
+make check
+```
+
+builds and runs the unit tests,
+
+
+```bash
+make install
+```
+
+to install as a shared library to 'default' locations (/usr/local/lib and /usr/local/include on linux)
+
+
+## Usage
+Example.cpp is a very basic Kafka Producer
+
+
+## API docs
+There isn't much code, if I get around to writing the other parts of the library I'll document it sensibly,
+for now have a look at the header file: /src/producer.hpp
+
+
+## Contact for questions
+
+Ben Gray, MediaSift Ltd.
+
+http://twitter.com/benjamg
+
+
|