Quantcast
Channel: Blog – Confluent
Viewing all 175 articles
Browse latest View live

A Comprehensive, Open Source REST Proxy for Kafka

$
0
0

As part of Confluent Platform 1.0 released about a month ago, we included a new Kafka REST Proxy to allow more flexibility for developers and to significantly broaden the number of systems and languages that can access Apache Kafka clusters. In this post, I’ll explain the REST Proxy’s features, how it works, and why we built it.

What is the REST Proxy and why do you need one?

The REST Proxy is an open source HTTP-based proxy for your Kafka cluster. The API supports many interactions with your cluster, including producing and consuming messages and accessing cluster metadata such as the set of topics and mapping of partitions to brokers. Just as with Kafka, it can work with arbitrary binary data, but also includes first-class support for Avro and integrates well with Confluent’s Schema Registry. And it is scalable, designed to be deployed in clusters and work with a variety of load balancing solutions.

We built the REST Proxy first and foremost to meet the growing demands of many organizations that want to use Kafka, but also want more freedom to select languages beyond those for which stable native clients exist today. However, it also includes functionality beyond traditional clients, making it useful for building tools for managing your Kafka cluster. See the documentation for a more detailed description of the included features.

A quick example

If you’ve used the Confluent Platform Quickstart to start a local test cluster, starting the REST Proxy for your local Kafka cluster should be as simple as running

$ kafka-rest-start

To use it with a real cluster, you only need to specify a few connection settings. The proxy includes good default settings so you can start using it without any need for customization.

The complete API provides too much functionality to cover in this blog post, but as an example I’ll show a couple of the most common use cases. To keep things language agnostic, I’ll just show the cURL commands. Producing messages is as easy as:

$ curl -i -X POST -H "Content-Type: application/vnd.kafka.avro.v1+json"
    --data '{
      "value_schema": "{\"type\": \"record\", \"name\": \"User\", \"fields\": [{\"name\": \"username\", \"type\": \"string\"}]}",
      "records": [
        {"value": {"username": "testUser"}},
        {"value": {"username": "testUser2"}}
      ]
    }' \
    http://localhost:8082/topics/avrotest

This sends an HTTP request using the POST method to the endpoint http://localhost:8082/topics/avrotest, which is a resource representing the topic avrotest. The content type, application/vnd.kaka.avro.v1+json, indicates the data in the request is for the Kafka proxy (application/vnd.kafka), contains Avro keys and values (.avro), using the first API version (.v1), and JSON encoding (+json). The payload contains a value schema to specify the format of the data (records with a single field username) and a set of records. Records can specify a key, value, and partition, but in this case we only include the value. The values are just JSON objects because the REST Proxy can automatically translate Avro’s JSON encoding, which is more convenient for your applications, to the more efficient binary encoding you want to store in Kafka.

The server will respond with:

HTTP/1.1 200 OK
Content-Length: 209
Content-Type: application/vnd.kafka.v1+json
Server: Jetty(8.1.16.v20140903)
{
  "key_schema_id": null,
  "value_schema_id": 1,
  "offsets": [
    {"partition": 0, "offset":0, "error_code": null, "error": null},
    {"partition": 0, "offset":1, "error_code": null, "error": null}
  ]
}

This indicates the request was successful (200 OK) and returns some information about the messages. Schema IDs are included, which can be used as shorthand for the same schema in future requests. Information about any errors for individual messages (error and error_code) are provided in case of failure, or are null in case of success. Successfully recorded messages include the partition and offset of the message.

Consuming messages requires a bit more setup and cleanup, but is also easy. Here we’ll consume the messages we just produced to the topic avrotest. Start by creating the consumer, which will return a base URI you use for all subsequent requests:

$ curl -i -X POST -H "Content-Type: application/vnd.kafka.v1+json" \
    --data '{"format": "avro", "auto.offset.reset": "smallest"}' \
    http://localhost:8082/consumers/my_avro_consumer

HTTP/1.1 200 OK
Content-Length: 121
Content-Type: application/vnd.kafka.v1+json
Server: Jetty(8.1.16.v20140903)

{
  "instance_id": "rest-consumer-1",
  "base_uri": "http://localhost:8082/consumers/my_avro_consumer/instances/rest-consumer-1"
}

Notice that we specified that we’ll consume Avro messages and that we want to read from the beginning of topics we subscribe to. Next, just GET a topic resource under that consumer to start receiving messages:

$ curl -i -X GET -H "Accept: application/vnd.kafka.avro.v1+json" \
    http://localhost:8082/consumers/my_avro_consumer/instances/rest-consumer-1/topics/avrotest

HTTP/1.1 200 OK
Content-Length: 134
Content-Type: application/vnd.kafka.avro.v1+json
Server: Jetty(8.1.16.v20140903)

[
  {
    "key": null,
    "value": {"username": "testUser"},
    "partition": 0,
    "offset": 0
  },
  {
    "key": null,
    "value": {"username": "testUser2"},
    "partition": 0,
    "offset": 1
  }
]

When you’re done consuming or need to shut down this instance, you should try to clean up the consumer so other instances in the same group can quickly pick up the partitions that had been assigned to this consumer:

$ curl -i -X DELETE \
    http://localhost:8082/consumers/my_avro_consumer/instances/rest-consumer-1

HTTP/1.1 204 No Content
Server: Jetty(8.1.16.v20140903)

This is only a short example of the API, but hopefully shows how simple it is to work with. The example above can quickly be adapted to any language with a good HTTP library in just a few lines of code. Using the API in real applications is just as simple.

How does the REST Proxy work?

I’ll leave a detailed explanation of the REST Proxy implementation for another post, but I do want to highlight some high level design decisions.

HTTP wrapper of Java libraries – At it’s core, the REST Proxy simply wraps the existing libraries provided with the Apache Kafka open source project. This includes not only the producer and consumer you would expect, but also access to cluster metadata and admin operations. Currently this means using some internal (but publicly visible) interfaces that may require some ongoing maintenance because that code has no compatibility guarantees; therefore the REST Proxy depends on specific versions of the Kafka libraries and the Confluent Platform packaging ensures the REST Proxy uses a compatible version. As the Kafka code standardizes some interfaces (and protocols) for these operations we’ll be able to rely on those public interfaces and therefore be less tied to particular Kafka versions.

JSON with flexible embedded data – Good JSON libraries are available just about everywhere, so this was an easy choice. However, REST Proxy requests need to include embedded data — the serialized key and value data that Kafka deals with. To make this flexible, we use vendor specific content types in Content-Type and Accept headers to make the format of the data explicit. We recommend using Avro to help your organization build a manageable and scalable stream data platform. Avro has a JSON encoding, so you can embed your data in a natural, readable way, as the introductory example demonstrated. Schemas need to be included with every request so they can be registered and validated against the Confluent Schema Registry and the data can be serialized using the more efficient binary encoding. To avoid the overhead of sending schemas with every request, API responses include the schema ID that the Schema Registry returns, which can subsequently be used in place of the full schema. If you opt to use raw binary data, it cannot be embedded directly in JSON, so the API uses a string containing the base64 encoded data.

Stateful consumers – Consumers are stateful and tied to a particular proxy instance. This is actually a violation of REST principles, which state that requests should be stateless and should contain all the information necessary to serve them. There are ways to make the REST Proxy layer stateless, e.g., by moving consumer state to a separate service, just as a separate database is used to manage state for a RESTful API. However, we felt the added complexity and the cost of extra network hops and latency wasn’t warranted in this case. The approach we use is simpler and the consumer implementation already provides fault tolerance: if one instance of the proxy fails, other consumers in the same group automatically pick up the slack until the failed consumers can be restarted.

Distributed and load balanced – Despite consumers being tied to the proxy instance that created them, the entire REST Proxy API is designed to be accessed via any load balancing mechanism (e.g. discovery services, round robin DNS, or software load balancing proxy). To support consumers, each instance must be individually addressable by clients, but all non-consumer requests can be handled by any REST Proxy instance.

Why a REST Proxy?

In addition to the official Java clients, there are more than a dozen community-contributed clients across a variety of languages listed on the wiki. So why bother writing the REST Proxy at all?

It turns out that writing a feature complete, high performance Kafka client is currently a pretty difficult endeavor for a few reasons:

Flexible consumer group model – Kafka has a consumer group model, as shown in the figure below, that generalizes a few different messaging models, including both queuing and publish-subscribe semantics. However, the protocol currently leaves much of the work to clients. This has some significant benefits for brokers, e.g., minimizing the work they do and making performance very predictable. But it also means Kafka consumers are “thick clients” that have to implement complex algorithms for features like partition load balancing and offset management. As a result, it’s common to find Kafka client libraries supporting only a fraction of the Java client’s functionality, sometime omitting high-level consumer support entirely.

Kafka Consumer Groups
Kafka’s consumer group model supports multiple consumers on the same topic, organized into groups of variable and dynamic size, and supports offset management. This is very flexible, scalable, and fault tolerant, but means non-Java clients have to implement more functionality to achieve feature parity with the Java clients.

Low-level networking – Even a small Kafka cluster on moderate hardware can support very high throughput, but writing a cluster to take advantage of this requires low level networking code. Producers need to efficiently batch messages to achieve high throughput while minimizing latency, manage many connections to brokers (usually using an async I/O API), and gracefully handle a wide variety of edge cases and errors. It’s a substantial time investment to write, debug, and maintain these libraries.

Protocol evolution – The 0.8 releases come with a promise of compatibility and a new versioned protocol to enable this. The community has also implemented a new approach to making user visible changes, both to catch compatibility problems before they occur and to act as documentation for users. As the protocol evolves and new features are added, client implementations need to add support for new features and remove deprecated ones. By leveraging the official Java clients, the REST Proxy picks up most of these changes and fixes for free. Some new features will require REST API changes, but careful API design that anticipates new features will help insulate applications from these changes.

The REST Proxy is a simple solution to these problems: instead of waiting for a good, feature-complete client for your language of choice, you can use a simple HTTP-based interface that should be easily accessible from just about any language.

But there were already a few REST proxies for Kafka out there, so why build another? Although the existing proxies were good — and informed the design of the Confluent REST Proxy — it was clear that they were written to address a few specific use cases. We wanted to provide a truly comprehensive interface to Kafka, which includes administrative actions in addition to complete producer and consumer functionality. We haven’t finished yet — today we provide read-only access to cluster metadata but plan to add support for administrative actions such as partition assignment — but soon you should be able to both operate and use your cluster entirely through the REST Proxy.

Finally, it’s worth pointing out that the goal of the REST Proxy is not to replace existing clients. For example, if you’re using C or Python and only need to produce or consume messages, you may be better served by the existing high quality librdkafka library for C/C++ or kafka-python library for Python. Confluent is invested in development of native clients for a variety of popular languages. However, the REST Proxy provides a broad-reaching solution very quickly and supports languages where no developer has yet taken on the task of developing a client.

Tradeoffs

Of course, there are some tradeoffs to using the REST Proxy instead of native clients for your language, but in many cases these tradeoffs are acceptable when you consider the capabilities you get with the REST Proxy.

The most obvious tradeoff is complexity: the REST Proxy is yet another service you need to run and maintain. We’ve tried to make this as simple as possible, but adding any service to your stack has a cost.

Another cost is in performance. The REST Proxy adds extra processing: clients construct and make HTTP requests, the REST Proxy needs to parse requests, transform data between formats both for produce and consume requests, and handle all the interaction with Kafka itself, at least doubling bandwidth usage. And the performance costs aren’t only as simple as doubling costs because the REST Proxy can’t use all the same optimizations that the Kafka broker can (e.g., sendfile is not an option for the proxy consumer responses). Still, the REST Proxy can achieve high throughput. I’ll discuss some benchmarking numbers in more detail in a future post, but the most recent version of the proxy achieves about ⅔ the throughput of the Java clients when producing binary messages and about ⅕ the rate when consuming binary messages. Some of this performance can be regained with higher performance CPUs on your REST Proxy instances because a significant fraction of the performance reduction is due to increased CPU usage (parsing JSON, decoding embedded content in produce requests or encoding embedded content in responses, serializing JSON responses).

The REST Proxy also exposes a very limited subset of the normal client configuration options to applications. It is designed to act as shared infrastructure, sitting between your Kafka cluster and many of your applications. Limiting configuration of many settings to the operator of the proxy in this multi-tenant environment is important to maintaining stability. One good example where this is critical is consumer memory settings. To achieve good throughput, consumer responses should include a relatively large amount of data to amortize per-request overheads over many messages. But the proxy needs to buffer that amount of data for each consumer, so it would be risky to give complete control of that setting to each application that creates consumers. One misbehaving consumer could affect all your applications. Future versions should offer more control, but these options need to be added to the API with considerable care.

Finally, unless you have a language-specific wrapper of the API, you won’t get an idiomatic API for your language. This can have a significant impact on the verbosity and readability of your code. However, this is also the simplest problem to fix, and at relatively low cost. For example, to evaluate and validate the API, I wrote a thin node.js wrapper of the API. The entire library clocks in at about 650 lines of non-comment non-whitespace code, and a simple example that ingests data from Twitter’s public stream and computes trending hashtags is only about 200 lines of code.

Conclusion

This first version of the REST Proxy provides basic metadata, producer, and consumer support using Avro or arbitrary binary data. We hope it will enable broader adoption of Kafka by making all of Kafka’s features accessible, no matter what language or technology stack you’re working with.

We’re just getting started with the REST Proxy and have a lot more functionality planned for future versions. You can get some idea of where we’re heading by looking at the features we’ve noted are missing right now, but we also want to hear from you about what features you would find most useful, either on our community mailing list, on the open source project’s issue tracker, or through pull requests.



Hands-free Kafka Replication: A lesson in operational simplicity

$
0
0

Building operational simplicity into distributed systems, especially for nuanced behaviors, is somewhat of an art and often best achieved after gathering production experience. Apache Kafka‘s popularity can be attributed in large part to its design and operational simplicity. As we add more knobs and features, we try to go back and rethink ways of simplifying complex behaviors.

One of the more nuanced features in Apache Kafka is its replication protocol. Tuning Kafka replication to work automatically, for varying size workloads on a single cluster, is somewhat tricky today. One of the challenges that make this particularly difficult is knowing how to prevent replicas from jumping in and out of the in sync replica list (aka ISR). What this means from a user’s perspective is that if a producer sends a batch of messages “large enough”, then this can cause several alerts to go off on the Kafka brokers. These alerts indicate that some topics are “under replicated” which means that data is not being replicated to enough number of brokers thereby increasing the probability of data loss should those replicas fail or die. So it is important that the “under replicated” partition count be monitored closely in a Kafka cluster. However, this alert should go off only when some broker fails, slows down or pauses and not when the producer writes data of varying sizes. This unexpected behavior and the false alarms are the source of a lot of manual operational overhead and churn. In this post, I discuss the root cause of this behavior and how we arrived at the fix.

Key takeaway – for the best operational experience, express configs in terms of what the user knows, not in terms of what the user has to guess.

KAFKA REPLICATION: 0 TO 60 IN 1 MINUTE

Every partition in a Kafka topic has a write-ahead log where the messages are stored and every message has a unique offset that identifies it’s position in the partition’s log.

fig1

Every topic partition in Kafka is replicated n times, where n is the replication factor of the topic. This allows Kafka to automatically failover to these replicas when a server in the cluster fails so that messages remain available in the presence of failures. Replication in Kafka happens at the partition granularity where the partition’s write-ahead log is replicated in order to n servers. Out of the n replicas, one replica is designated as the leader while others are followers. As the name suggests, the leader takes the writes from the producer and the followers merely copy the leader’s log in order.

fig2

The fundamental guarantee a log replication algorithm must provide is that if it tells the client a message is committed, and the leader fails, the newly elected leader must also have that message. Kafka gives this guarantee by requiring the leader to be elected from a subset of replicas that are “in sync” with the previous leader or, in other words, caught up to the leader’s log. The leader for every partition tracks this in-sync replica (aka ISR) list by computing the lag of every replica from itself. When a producer sends a message to the broker, it is written by the leader and replicated to all the partition’s replicas. A message is committed only after it has been successfully copied to all the in-sync replicas. Since the message replication latency is capped by the slowest in-sync replica, it is important to quickly detect slow replicas and remove them from the in-sync replica list. The details of Kafka’s replication protocol are somewhat nuanced and this blog is not intended to be an exhaustive discussion of the topic. You can read more about how Kafka replication works here. For the sake of this discussion, we’ll focus on operability of the replication protocol.

WHAT DOES IT MEAN FOR A REPLICA TO be caught up to the leader?

I referred to a replica that has not “caught up” to the leader’s log as possibly being marked as an out-of-sync replica. Let me explain what being caught up really means with the help of an example. Let’s take an example of a single partition topic foo with a replication factor of 3. Assume that the replicas for this partition live on brokers 1, 2 and 3 and that 3 messages have been committed on topic foo. Replica on broker 1 is the current leader and replicas 2 and 3 are followers and all replicas are part of the ISR. Also assume that replica.lag.max.messages is set to 4 which means that as long as a follower is behind the leader by not more than 3 messages, it will not be removed from the ISR. And replica.lag.time.max.ms is set to 500 ms which means that as long as the followers send a fetch request to the leader every 500 ms or sooner, they will not be marked dead and will not be removed from the ISR.

fig3
Now let’s say, the producer sends the next 1 message to the leader and at the same time follower broker 3 goes into a GC pause, their logs would look like this –

fig4 Since broker 3 is in the ISR, the latest message is not considered to be committed until either broker 3 is removed from the ISR or catches up to the leader’s log end offset. Note that since broker 3 is less than replica.lag.max.messages=4 messages behind the leader, it does not qualify to be removed from the ISR. In this case, it means follower broker 3 needs to catch up to offset 3 and if it did, then it has fully “caught up” to the leader’s log. Let’s assume that broker 3 comes out of its GC pause within 100ms and catches up to the leader’s log end offset. In this state, their logs would look like this-

fig5

WHAT CAUSES A REPLICA TO BE OUT-OF-SYNC WITH THE LEADER?

A replica can be out-of-sync with the leader for several reasons-

  • Slow replica: A follower replica that is consistently not able to catch up with the writes on the leader for a certain period of time. One of the most common reasons for this is an I/O bottleneck on the follower replica causing it to append the copied messages at a rate slower than it can consumer from the leader.
  • Stuck replica: A follower replica that has stopped fetching from the leader for a certain period of time. A replica could be stuck either due to a GC pause or because it has failed or died.
  • Bootstrapping replica: When the user increases the replication factor of the topic, the new follower replicas are out-of-sync until they are fully caught up to the leader’s log.

A replica is considered to be out-of-sync or lagging when it falls “sufficiently” behind the leader of the partition. In Kafka 0.8.2, a the replica’s lag is measured either in terms of number of messages it is behind the leader (replica.lag.max.messages) or the time for which the replica has not attempted to fetch new data from the leader (replica.lag.time.max.ms). The former is used to detect slow replicas while the latter is used to detect halted or dead replicas.

HOW DO YOU DETERMINE THAT A REPLICA IS LAGGING?

This model of detecting out-of-sync stuck replicas works well in all cases. It tracks the time for which a follower replica has not sent a fetch request to the leader, indicating it is dead. On the other hand, the model of detecting out-of-sync slow replicas using the number of messages only works well if you set these parameters for a single topic or multiple topics with homogeneous traffic patterns, but we’ve found that it does not scale to the variety of workloads across all topics in a production cluster.

Building on top of my previous example, if topic foo gets data at the rate of 2 msg/sec where a single batch received on the leader normally never exceeds 3 messages, then you know that replica.lag.max.messages can be set to 4 for that topic. Why? Because after the largest batch is appended to the leader and before the follower replicas copy those messages, the follower’s logs will be behind the leader by no more than 3 messages. At the same time, if the follower replicas for topic foo start lagging behind the leader consistently by more than 3 messages, you want the leader to remove the slow follower replica and prevent the message write latency from increasing.

Which is essentially the goal of replica.lag.max.messages – being able to detect replicas that are consistently out-of-sync with the leader. However, now, let’s say, the traffic on the same topic increases organically or due to a spike and the producer ends up sending a batch of 4 messages, equal to the configured value for replica.lag.max.messages=4. At that instant, both follower replicas, will be considered out-of-sync with the leader and will be pushed out of the ISR.

fig6However, since both follower replicas are alive, they will catch up to the leader’s log end offset in the next fetch request and be added back to the ISR. The same process will repeat if the producer continues to send relatively large batch of messages to the leader. This demonstrates the case when follower replicas shuttle in and out of the ISR unnecessarily triggering false alerts.

fig7

This points to the core problem with replica.lag.max.messages. It expresses the replication configs based on a value that the user has to guess and doesn’t know for sure at the time of configuration – the incoming traffic on Kafka topics!

ONE CONFIG TO RULE THEM ALL

What we realized is that there is only thing that really matters in order to detect either a stuck or a slow replica and that’s the time for which a replica has been out-of-sync with the leader. Removing the lag definition in terms of number of messages gets rid of the need to guess the right value based on the expected traffic for the topic. Now there is only one value you need to configure on the server which is replica.lag.time.max.ms. The interpretation of this has changed to be the time for which a replica has been out-of-sync with the leader. Stuck or failed replicas are detected the same way as before – if a replica fails to send a fetch request for longer than replica.lag.time.max.ms, it is considered dead and is removed from the ISR. The mechanism of detecting slow replicas has changed – if a replica starts lagging behind the leader for longer than replica.lag.time.max.ms, then it is considered too slow and is removed from the ISR. So even if there is a spike in traffic and large batches of messages are written on the leader, unless the replica consistently remains behind the leader for replica.lag.time.max.ms, it will not shuffle in and out of the ISR.

This new model for detecting out-of-sync replicas puts an upper bound on the message commit latency and also removes the need for any guesswork.

how do I GET this?

This change will available in the next version of the Confluent Platform. We’d like to thank Aditya Auradkar for contributing this enhancement to Kafka. If you’d like to get involved and contribute to Kafka, sign up for the mailing list or check out some newbie JIRAs. If you like working on Kafka and are interested in working in the real-time streaming space, Confluent is hiring!


Real-time full-text search with Luwak and Samza

$
0
0

This is an edited transcript of a talk given by Alan Woodward and Martin Kleppmann at FOSDEM 2015.

Traditionally, search works like this: you have a large corpus of documents, and users write ad-hoc queries to find documents within that corpus. Documents may change from time to time, but on the whole, the corpus is fairly stable.

However, with fast-changing data, it can be useful to turn this model on its head, and search over a stream of documents as they appear. For example, companies may want to detect whenever they are mentioned in a feed of news articles, or a Twitter user may want to see a continuous stream of tweets for a particular hashtag.

In this talk, we describe open source tools that enable search on streams: Luwak is a Lucene-based library for running many thousands of queries over a single document, with optimizations that make this process efficient. Samza is a stream processing framework based on Kafka, allowing real-time computations to be distributed across a cluster of machines. We show how Luwak and Samza can be combined into an efficient and scalable streaming search engine.

Searching over streams with Luwak & Samza

In this talk we’re going to discuss some work that we’ve been doing in the area of full-text search on streams. Perhaps you already know about normal search engines like Elasticsearch and Solr, but as we’ll see, searching on streams is quite a different problem, with some interesting challenges.

Searching on streams becomes important when you’re dealing with real-time data that is rapidly changing. We’ll see some examples later of when you might need it.

What is a stream?

But first of all, we should define what we mean with a stream. For our purposes, we’ll say that a stream is an append-only, totally ordered sequence of records (also called events or messages). For example, a log file is a stream: each record is a line of text with some structure, perhaps some metadata like a timestamp or severity, perhaps an exception stack trace. Every log record is appended to the end of the file.

Appending to a log, and tailing it

There are a few ways you can read the content of a stream. For example, you can start at the beginning of the file and read the entire file sequentially. Or you can use tail -f to watch the file for any new records that are appended, and be notified when new data appears.

We call a process that writes to a stream a “producer”, and a process that reads from the stream a “consumer”.

How do you search a stream?

Now say you’ve got some data in a stream, such as a log file, and you want to do full-text search on it. How do you go about doing that?

Put the contents of a log file in an index

The traditional approach is to load everything into a big search index, perhaps something like Elasticsearch or Solr. ELK (Elasticsearch, Logstash and Kibana) is a currently trendy way of setting this up. That way you have the entire history of the stream searchable, and people can write any queries they want to search the index.

Partitioned indexes for different time periods

But what happens as new records are appended to the stream? You need to add them to an index in order to make them searchable. For example, you could imagine creating different indexes for different time periods: one for historical data, one for yesterday, one for the last hour, one for the last minute…

And this is basically what people mean when they talk about “near-real-time” search: create an index for documents that appeared very recently, and send any queries to that index as well as the older historical indexes.

Let’s talk about some examples of this kind of search in practice.

Example: Twitter search

Take Twitter, for example. If you type something in the search box, you’ll see a list of tweets that match your search query, ranked by recency. The index includes all public tweets ever written, broken down my time period, similar to the diagram above.

But if you stay on that page for a while, notice that something happens: a bar appears at the top of the page, saying that there are new results for your query. What happened here? When you typed your search query, it seems that Twitter didn’t forget about the query the moment they returned the results to you. Rather, they must have remembered the query, and continued to search the stream of tweets for any new matches for your query. When new matches appear in the stream, they send a notification to your browser.

In this case, the stream we’re searching is Twitter’s so-called firehose of Tweets. I don’t know how they’ve implemented that. Perhaps they group tweets into batches — say, create an index for 10 seconds worth of tweets, and then run the queries from all open search sessions against that index. But somehow they are doing full-text search on a stream.

Example: Google alerts

Another example is Google Alerts. This is a feature of Google where you can register some search queries with their system, and they send you an email notification when new web pages matching your query are published. For example, you might set up an alert for your name or company name, so that you find out when people write stuff about you.

Google internally has a stream of new web pages being discovered by its crawler, and Google Alerts allows you to register a query against that stream. Google remembers the query, and runs the query against every new document that is discovered and added to its index.

Comparing after-the-fact search and streaming search

So it seems that we can divide search into two categories:
 

  • In one case, you put all the documents in a big index, and people can search that index by writing ad-hoc queries. We could call that “after-the-fact search”, because it’s searching a repository of historical documents that we received at some point in the past.
  • In the other case, you register the queries in advance, and then the system checks each document that appears on a stream, to see whether it matches any of the registered queries. This is streaming search.

 
It often makes sense to combine these two: for example, in the Twitter case, both types of search are happening. You first get after-the-fact search results from the last 7 days, but while you have the search page open, your query is also registered for a stream search, so that you can follow the stream of tweets that match your query.

You might have seen this pattern in Elasticsearch Percolator, for example. The streaming search approach we’re describing here is similar to Percolator, but we think that it will scale better.

One document, many queries

So, how do you actually implement streaming search? Well, we saw earlier that for near-real-time search, you construct a small index for recently added documents. We can take that approach to the extreme: for each new document that appears on a stream, we create a new index, containing just that one document. Then we can run through all of the registered queries, test each query against the one-document index, and output a list of all the queries that match the new document. A downstream system can then take those matches and notify the owners of the queries.

Does it scale?

However, the question is: how efficient is this going to be when you have lots of queries, or very complex queries? If you only have a few hundred registered queries, you can pretty quickly run through all of those queries for every new document on the stream — but if you have hundreds of thousands of queries, it can get quite slow.

Also, if you have very complex queries, even just executing a single query can take a long time. I (Alan) work with some clients who provide media monitoring services (also known as clipping companies). They collect feeds of newspaper articles and other news from around the world, and their clients are companies who want to know whenever they are mentioned in the news. The media monitoring companies construct one big query for each client, and those queries can become really huge — a query might be hundreds of kilobytes long! They contain a large number of terms, have lots of nested boolean operators, and lots of exclusions (negated search terms).

To give just one example, in the UK there’s a magazine called “Which?”. If you simply search for the term “which”, you match a huge number of documents, since that’s such a common word in English. They have to construct really complex queries to filter out most of the noise.

So, if you have a large number of queries, or very complex queries, the streaming search becomes slow. We need to find ways of optimizing that. Observe this: the fastest query is a query that you never execute. So, if we can figure out which queries are definitely not going to match a given document, we can skip those queries entirely, and potentially save ourselves a lot of work.

Flax Luwak

Which brings us to Luwak, a library that we (Flax) wrote in order to do efficient streaming search. Luwak is open source and builds upon Apache Lucene. It works the other way round from a normal search index: with a normal index, you first add documents to the index, and then you query it. Luwak turns this on its head: you first register queries with Luwak, and then match documents against them. Luwak tells you which of the registered queries match the document.

Let’s go a bit into the detail of how Luwak optimizes this matching process.

Index of queries

As I said, we want some way of figuring out which queries are definitely not going to match a document, so that we don’t need to bother executing those queries. In order to do this, we can do something unusual: we can index the queries! In search engines you normally index documents, but in this case we’re going to index the queries.

Let’s say we have three registered queries: Q1 is “WHEELS” NEAR “BUS”, Q2 is “WHEELS” NEAR “CAR”, and Q3 is “WHEELS” OR “BUMPERS”. First observe this: in a conjunction query (that is, a query like A AND B AND C), all the search terms must appear in the document for the query to match. (An operator like NEAR is a specialized form of AND that has an additional proximity restriction.) For example, a document must contain both “wheels” and “bus” in order to match Q1; if a document doesn’t contain the word “bus”, there’s no chance it can match Q1.

That means, we can arbitrarily pick one term from a conjunction and check whether the document contains that term. If the document doesn’t contain the term, we can be sure that the document won’t match the conjunction either.

On the other hand, a disjunction query (with an OR operator, like Q3 for example) matches if any of the search terms appear in the document. For example, if a document doesn’t contain “wheels”, it may nevertheless match Q3 if it contains “bumpers”. In this case, we must extract all of the terms from the disjunction; if any one of those terms appears in the document, we have to test the full query.

We can now take those terms that we extracted from queries (for example “bus” from Q1, “car” from Q2, and “bumpers” and “wheels” from Q3), and build an index of those terms. As I said, this is an index of queries, not of documents. The index is a dictionary from terms to queries: it maps terms to queries containing that term.

Document disjunction

Now that we’ve taken all our registered queries and indexed them, we can move on to the next step: processing the stream of documents. For each document, we want to find all the queries that match. How do we go about doing this?

The trick is to take each document, and turn it into a query. (Previously we created an index of queries. Now we’re turning a document into a query. How upside-down!) Namely, we take all the words (terms) that appear in the document, and construct a disjunction (OR) query from all of those words. Intuitively, this is saying: “find me all the queries that match any of the words in this document”.  Creating an inverted index from a single document automatically gives us this list of terms.

Selecting candidate queries

Now that we have created an index of queries, and turned a document into a query, we can figure out which queries match the document. First, we run the document disjunction query against the index of queries. This will tell us which queries may match the document.

In our example, we created the query index by extracting the term “bus” from Q1, the term “car” from Q2, and the terms “bumpers” and “wheels” from Q3. Also, we turned the document “The wheels on the bus go round and round” into the disjunction query:

“and” OR “bus” OR “go” OR “on” OR “round” OR “the” OR “wheels”

Running that disjunction query against the query index, we get a hit on the terms “bus” (Q1) and “wheels” (Q3), but the terms “bumpers” (Q3) and “car” (Q2) don’t appear in the document. Therefore we can conclude that Q1 and Q3 might match the document, but Q2 definitely doesn’t match the document.

The next step is then to run queries Q1 and Q3 against the document index, to see whether they really do match. But we don’t need to run Q2, because we’ve already established that it definitely doesn’t match.

This whole process of indexing queries may seem a bit complicated, but it is a really powerful optimization if you have a large number of queries. It can cut out 99% of the queries you would otherwise have to execute, and thus massively speed up searching on streams. As I said, the fastest query is one that you never even execute.

Query decomposition

Besides indexing queries, there are other optimizations we can make. One particular optimization that we’ve found useful: if you have a big query that contains an OR operator at the top level, you can break that big query into smaller ones. That is especially useful if one of the subqueries is simple (fast to execute), and another one is complex (slow to execute).

Say we have a document that will match on the simple subquery, but not on the complex one.  In the normal case, the whole query is run against the document, so we still pay the price for executing the complex subquery, even though it doesn’t match.  If we decompose it into its constituent parts, however, then only the simple subquery will be selected, and we can avoid the performance hit of running the complex one.

Term frequency analysis and phrase query analysis

We said earlier that when you’re indexing the queries, you can make some arbitrary choices about which terms to extract. For example, for the query “car” AND “bumpers”, you could choose either “car” or “bumpers” as the term to use in the query index. Which one should you choose?

It’s helpful to know how often each term occurs in your documents. For example, perhaps “car” is quite a common term, but “bumpers” is much more rare. In that case, it would be better to use “bumpers” in the query index, because it’s less likely to appear in documents. Only the small number of documents containing the term “bumpers” would then need to be matched against the query “car” AND “bumpers”, and you save yourself the effort of executing the query for the large number of documents that contain “car” but not “bumpers”.

Query term extraction

Another, more advanced optimization technique considers several different possibilities of extracting terms from queries. Take the query tree above, containing five terms and four boolean operators. Remember the rule for extracting terms that we established earlier: extract any one of the children of a conjunction (AND), but extract all of the children of a disjunction (OR). This means there are three different combinations of terms that you could extract from the above query tree:

Three ways of extracting terms from a query

Say you have a document “term1 term2 term3”: this document does not match the query (because neither of the required term4 or term5 appears). However, in the first two combinations above (term1 and term2 extracted, or term1 and term3 extracted), the document would nevertheless be selected to be matched against the query. In the third combination above (term4 and term5 extracted), the document wouldn’t be selected, because we can tell from the query index that it is definitely not going to match the query.

Can we make the query pre-selection more precise? Yes, we can! Rather than just extracting one (arbitrary) set of terms from the query, we can extract several sets of terms from the same query, like the three above, and index them into separate fields (let’s call them _a, _b and _c). You then run your document disjunction against all those fields, and the set of queries you need to run is the intersection of those results — a conjunction of disjunctions, if you like.

The document “term1 term2 term3”, which we previously turned into a simple disjunction of terms, now turns into something like this:

_a:(term1 OR term2 OR term3) AND
_b:(term1 OR term2 OR term3) AND
_c:(term1 OR term2 OR term3)

The first two terms of the conjunction match, but the third doesn’t, and so we don’t select this query. It’s still an approximation — you still need to execute the full query to be sure whether it matches or not — but with these optimizations you can further reduce the number of queries you need to execute.

Fortunately, Luwak has implemented all of these optimizations already, and they’re available for you to use today.

However, as described so far, Luwak runs on a single machine. At some point, you may have so many queries or such high throughput of documents that a single machine is not enough, even after you have applied all of these optimizations. Which brings us to the second half of this talk: scaling stream search across a cluster of machines.

[At this point, Martin took over from Alan]

Kafka and Samza

Rather than inventing our own distributed computing framework — which would be likely to go wrong, because distributed systems are hard — we’re going to build on a robust foundation. We’re going to use Apache Kafka and Apache Samza, two open source projects that originated at LinkedIn.

I’ll start by giving a bit of background about Kafka, and then talk about how we embedded Luwak inside Samza in order to scale out search on streams.

A partitioned stream in Kafka

Kafka is a kind of message broker or message queue — that is, it takes messages that originate in one process (a producer), and delivers them to another process (a consumer). It does so in a scalable, fault-tolerant manner.

The way it works is simple but remarkably effective. You can imagine Kafka as one big, append-only file. Whenever a producer wants to send a new message to a stream, it simply appends it to the end of the file. That’s the only way how you can write to Kafka: by appending to the end of a file.

A consumer is like a tail -f on that file, just like what we described at the beginning of this talk. Each consumer reads the messages in the file sequentially, and each consumer has a current position or offset in this file. Thus, it knows that all messages before that position have already been read, and all messages after that position have not yet been read. This makes Kafka very efficient: the brokers don’t have to keep track of which consumer has seen which messages, because the consumers themselves just need to store their current position.

In order to scale across multiple machines, a Kafka stream is also partitioned. That means, there’s not just one append-only file, but several. Each partition is completely independent from the others, so different partitions can live on different machines.

Replication over Kafka broker nodes

In addition, Kafka provides replication, i.e. maintaining a copy of the same data on multiple machines. This is important for fault tolerance — so that if one machine dies, you don’t lose any data, and everything keeps running.

Kafka does this using a leader/follower model. Each partition of a stream has a leader on one broker node, and a configurable number of followers on other broker nodes. All the new messages for a partition go to its leader, and Kafka replicates them from the leader to the followers. If a broker node goes down, and it was the leader for some partition, then one of the followers for that partition becomes the new leader.

Every message in Kafka has a key and a value (which can be arbitrary byte strings). The key is used for two different things: firstly, it determines which partition the message should go to (we make sure that all the messages with the same key go to the same partition, by choosing the partition based on a hash of the key). Secondly, it is used for compaction.

Kafka changelog compaction

Compaction is an exception to Kafka’s otherwise strict append-only model. You don’t have to use compaction, but if you do turn it on, then Kafka keeps track of the keys in the stream. And if there are several messages with the same key, then Kafka is allowed to throw away older messages with that key — only the newest message for a given key is guaranteed to be retained.

In the picture above, there are originally three messages with key A, and compaction discards two of them. In effect, this means that later messages can “overwrite” earlier messages with the same key. This overwriting doesn’t happen immediately, but at some later time: compaction is done in a background thread, a bit like garbage collection.

A Kafka stream with compaction is thus similar to a database with a key-value data model. If a key is never overwritten, it is never discarded, so it stays in the stream forever. With compaction, we can thus keep a complete history of key-value pairs:

Keeping complete history of events in Kafka

Without compaction, the stream would keep growing forever (the size of the stream is proportional to the number of messages ever sent). But with compaction, the size of the stream is proportional to the number of distinct keys — just like a database. If you can imagine storing all the keys and values in a database, you can equally well store all the keys and values in a Kafka stream.

Why is this useful? Well, if we want to use Luwak in a reliable manner, there is a problem we need to solve: when you register a query with Luwak, it is only kept in memory. Thus, whenever the Luwak process is restarted, it needs to reload its list of queries from stable storage into memory.

You could use a database for this, but using a Kafka stream has a big advantage: Luwak can consume the stream of queries, so it gets notified whenever someone registers a new query, modifies a registered query, or unregisters (deletes) a query. We simply use the query ID as the message key, and the query string as the message value (or a null value when a query is unregistered). And stream compaction ensures that the query stream doesn’t get too big.

Re-processing historical events from stream

Now, whenever Luwak starts up, it can jump to the very beginning of the queries stream, and consume it from beginning to end. All queries in that stream are loaded into memory, so that Luwak knows what queries it should apply to documents. Only once it has finished consuming the stream, Luwaks starts processing documents and matching them against queries. We call the query stream a bootstrap stream, because it’s used to bootstrap (initialize) the in-memory state of the stream consumer.

This brings us to Samza, a framework for writing stream processing jobs on top of Kafka. A basic Samza job is very simple: you write a bit of code (implementing a Java interface called StreamTask) and tell it which stream you want to consume, and Samza calls the process() method on your code for every message that is consumed from the input stream. The code can do whatever it wants, including sending messages to an output stream.

Samza takes a Kafka stream an input, produces another as output

As the input stream from Kafka is split into partitions, Samza creates a separate StreamTask for each partition, and each StreamTask processes the messages in its corresponding partition sequentially. Although a StreamTask only processes input from one partition, it can send output to any partition of its output streams.

This partitioning model allows a Samza job to have two or more input streams, and “join” them together:

Joining streams in Samza

By default, if a job has two input streams (say A and B), Samza sends partition 1 of stream A and partition 1 of stream B to the same StreamTask 1; it sends partition 2 of A and partition 2 of B to the same StreamTask 2; and so on. This is illustrated in the picture above. Note this only really works if both input streams have the same number of partitions.

This allows the stream join to scale, by partitioning both input streams in the same way. For example, if both streams are partitioned by user ID (i.e. using the user ID as the Kafka message key), then you can be sure that all the messages for a particular user ID on both input streams are routed to the same StreamTask. That StreamTask can then keep whatever state it needs in order to join the streams together.

How do we bring full-text search on streams into this processing model?

Integration of Luwak with Samza

Alan and I hacked together a proof-of-concept called Samza-Luwak to test using Luwak inside a Samza job. It works as follows:

There are two input streams (Kafka topics): one for queries, and one for documents. The query stream is a bootstrap stream with compaction, as described above. Whenever a user wants to register, modify or unregister a query, they send a message to the queries stream. The Samza job consumes this stream, and whenever a message appears, it updates Luwak’s in-memory roster of queries.

The documents stream contains the things that should be matched by the queries (the tweets if you’re building Twitter search, the web pages from the crawler if you’re building Google Alerts, etc). The Samza job also consumes the documents stream, and whenever a document appears, it is matched against the index of registered queries, as described previously. It then sends a message to an output stream, indicating which queries matched.

How do we distribute this matching process across multiple machines? The problem is that Samza’s default partitioning model actually doesn’t do what we need. As I said previously, Samza by default sends partition 1 of all the input streams to task 1, partition 2 of all the input streams to task 2, and so on:

Joining by co-partitioning two streams

This is a good partitioning model if you’re doing an equi-join, because you can set up the partitioning of the input streams such that each tasks only needs to know about its own input partitions, and can ignore all the other partitions. This allows you to increase parallel processing and scale the computation simply by creating more partitions.

However, full-text search is different. We’re not doing an equi-join on a particular field, we’re trying to find matches involving arbitrarily complicated boolean expressions. In general, we don’t know in advance which documents are going to match which queries. (The query index tells us approximately which queries might match, but it’s not precise enough to use for partitioning.)

We still want to partition the query and document streams, because that will allow the system to scale. But we also want to be able to match every document against every possible query. In other words, we need to make sure that every query partition is joined with every document partition:

Cartesian product join

If you think about it, what we need is a cartesian product of query partitions and document partitions. We want to create a separate StreamTask for every possible combination of a query partition and a document partition. For example, in the picture above, StreamTask 8 is responsible for handling query partition 4 and document partition 2.

This gives us exactly the semantics we need: every query is sent to multiple tasks (one task per document partition), and conversely, every document is sent to multiple tasks (one task per query partition). Each task can independently do its work on its own partitions, and afterwards you just need to combine all the matches for each document. The dataflow is similar to the scatter-gather approach you get in distributed search engines.

Unfortunately, this mode of streaming joins is not yet supported in Samza, but it’s being worked on (you can track it under SAMZA-353 if you’re interested). Once this feature is in place, you’ll be able to perform full-text search on streams at arbitrary scale, simply by adding new partitions and adding more machines to the cluster. Combining the clever indexing of Luwak with the scalability of Kafka and Samza — isn’t that cool?

If you want to play with Samza, there’s a quickstart project “hello-samza”, and you can find our proof-of-concept integration of Samza and Luwak on Github.

Alan Woodward is a director of Flax, a consultancy specializing in open source search engines. He is a committer on Lucene/Solr, and developer of Luwak. He previously worked on the enterprise search product Fast ESP.

Martin Kleppmann is a committer on Apache Samza and author of the upcoming O’Reilly book Designing Data-Intensive Applications. He previously co-founded Rapportive, which was acquired by LinkedIn.


Bottled Water: Real-time integration of PostgreSQL and Kafka

$
0
0

Summary: Confluent is starting to explore the integration of databases with event streams. As part of the first step in this exploration, Martin Kleppmann has made a new open source tool called Bottled Water. It lets you transform your PostgreSQL database into a stream of structured Kafka events. This is tremendously useful for data integration.

Writing to a database is easy, but getting the data out again is surprisingly hard.

Of course, if you just want to query the database and get some results, that’s fine. But what if you want a copy of your database contents in some other system — for example, to make it searchable in Elasticsearch, or to pre-fill caches so that they’re nice and fast, or to load it into a data warehouse for analytics, or if you want to migrate to a different database technology?

If your data never changed, it would be easy. You could just take a snapshot of the database (a full dump, e.g. a backup), copy it over, and load it into the other system. The problem is that the data in the database is constantly changing, and so the snapshot is already out-of-date by the time you’ve loaded it. Even if you take a snapshot once a day, you still have one-day-old data in the downstream system, and on a large database those snapshots and bulk loads can become very expensive. Not really great.

So what do you do if you want a copy of your data in several different systems?

One option is for your application to do so-called “dual writes”. That is, every time your application code writes to the database, it also updates/invalidates the appropriate cache entries, reindexes the data in your search engine, sends it to your analytics system, and so on:

Application-managed dual writes

However, as I explain in one of my talks, the dual-writes approach is really problematic. It suffers from race conditions and reliability problems. If slightly different data gets written to two different datastores (perhaps due to a bug or a race condition), the contents of the datastores will gradually drift apart — they will become more and more inconsistent over time. Recovering from such gradual data corruption is difficult.

If you rebuild a cache or index from a snapshot of a database, that has the advantage that any inconsistencies get blown away when you rebuild from a new database dump. However, on a large database, it’s slow and inefficient to process the entire database dump once a day (or more frequently). How could we make it fast?

Typically, only a small part of the database changes between one snapshot and the next. What if you could process only a “diff” of what changed in the database since the last snapshot? That would also be a smaller amount of data, so you could take such diffs more frequently. What if you could take such a “diff” every minute? Every second? 100 times a second?

When you take it to the extreme, the changes to a database become a stream. Every time someone writes to the database, that is a message in the stream. If you apply those messages to a database in exactly the same order as the original database committed them, you end up with an exact copy of the database. And if you think about it, this is exactly how database replication works.

The replication approach to data synchronization works much better than dual writes. First, you write all your data to one database (which is probably what you’re already doing anyway). Next, you extract two things from that database:

  1. a consistent snapshot at one point in time, and
  2. a real-time stream of changes from that point onwards.

You can load the snapshot into the other systems (for example your search indexes or caches), and then apply the real-time changes on an ongoing basis. If this pipeline is well tuned, you can probably get a latency of less than a second, so your downstream systems remain very almost up-to-date. And since the stream of changes provides ordering of writes, race conditions are much less of a problem.

This approach to building systems is sometimes called Change Data Capture (CDC), though the tools for doing it are currently not very good. However, at some companies, CDC has become a key building block for applications — for example, LinkedIn built Databus and Facebook built Wormhole for this purpose.

I am excited about change capture because it allows you to unlock the value in the data you already have. You can feed the data into a central hub of data streams, where it can readily be combined with event streams and data from other databases in real-time. This approach makes it much easier to experiment with new kinds of analysis or data format, it allows gradual migration from one system to another with minimal risk, and it is much more robust to data corruption: if something goes wrong, you can always rebuild a datastore from the snapshot and the stream.

Using change capture to drive derived data stores

Getting the real-time stream of changes

Getting a consistent snapshot of a database is a common feature, because you need it in order to take backups. But getting a real-time stream of changes has traditionally been an overlooked feature of databases. Oracle GoldenGate, the MySQL binlog, the MongoDB oplog or the CouchDB changes feed do something like this, but they’re not exactly easy to use correctly. More recently, a few databases such as RethinkDB or Firebase have oriented themselves towards real-time change streams.

However, today we will talk about PostgreSQL. It’s an old-school database, but it’s good. It is very stable, has good performance, and is surprisingly full-featured.

Until recently, if you wanted to get a stream of changes from Postgres, you had to use triggers. This is possible (see below), but it is fiddly, requires schema changes and doesn’t perform very well. However, Postgres 9.4 (released in December 2014) introduced a new feature that changes everything: logical decoding (which I explain in more detail below).

With logical decoding, change data capture for Postgres suddenly becomes much more appealing. So, when this feature was released, I set out to build a change data capture tool for Postgres that would take advantage of the new facilities. Confluent sponsored me to work on it (thank you Confluent!), and today we are releasing an alpha version of this tool as open source. It is called Bottled Water.

Bottled Water: Data streams freshly bottled at source

Introducing Bottled Water

Logical decoding takes the database’s write-ahead log (WAL), and gives us access to row-level change events: every time a row in a table is inserted, updated or deleted, that’s an event. Those events are grouped by transaction, and appear in the order in which they were committed to the database. Aborted/rolled-back transactions do not appear in the stream. Thus, if you apply the change events in the same order, you end up with an exact, transactionally consistent copy of the database.

The Postgres logical decoding is well designed: it even creates a consistent snapshot that is coordinated with the change stream. You can use this snapshot to make a point-in-time copy of the entire database (without locking — you can continue writing to the database while the copy is being made), and then use the change stream to get all writes that happened since the snapshot.

Bottled Water uses these features to copy all the data in a database, and encodes it in the efficient binary Avro format. The encoded data is sent to Kafka — each table in the database becomes a Kafka topic, and each row in the database becomes a message in Kafka.

Once the data is in Kafka, you can easily write a Kafka consumer that does whatever you need: send it to Elasticsearch, or populate a cache, or process it in a Samza job, or load it into HDFS with Camus… the possibilities are endless.

Why Kafka?

Kafka is a messaging system, best known for transporting high-volume activity events, such as web server logs and user click events. In Kafka, such events are typically retained for a certain time period and then discarded. Is Kafka really a good fit for database change events? We don’t want database data to be discarded!

In fact, Kafka is a perfect fit — the key is Kafka’s log compaction feature, which was designed precisely for this purpose. If you enable log compaction, there is no time-based expiry of data. Instead, every message has a key, and Kafka retains the latest message for a given key indefinitely. Earlier messages for a given key are eventually garbage-collected. This is quite similar to new values overwriting old values in a key-value store.

Bottled Water identifies the primary key (or replica identity) of each table in Postgres, and uses that as the key of the messages sent to Kafka. The value of the message depends on the kind of event:

  • For inserts and updates, the message value contains all of the row’s fields, encoded as Avro.
  • For deletes, the message value is set to null. This causes Kafka to remove the message during log compaction, so its disk space is freed up.

With log compaction, you don’t need one system to store the snapshot of the entire database and another system for the real-time messages — they can live perfectly well within the same system. Bottled Water writes the initial snapshot to Kafka by turning every single row in the database into a message, keyed by primary key, and sending them all to the Kafka brokers. When the snapshot is done, every row that is inserted, updated or deleted similarly turns into a message.

If a row frequently gets updated, there will be many messages with the same key (because each update turns into a message). Fortunately, Kafka’s log compaction will sort this out, and garbage-collect the old values, so that we don’t waste disk space. On the other hand, if a row never gets updated or deleted, it just stays unchanged in Kafka forever — it never gets garbage-collected.

Having the full database dump and the real-time stream in the same system is tremendously powerful. If you want to rebuild a downstream database from scratch, you can start with an empty database, start consuming the Kafka topic from the beginning, and scan through the whole topic, writing each message to your database. When you reach the end, you have an up-to-date copy of the entire database. What’s more, you can continue keeping it up-to-date by simply continuing to consume the stream. Building alternative views onto your data was never easier!

The idea maintaining a copy of your database in Kafka surprises people who are more familiar with traditional enterprise messaging and its limitations. Actually, this use case is exactly why Kafka is built around a replicated log abstraction: it makes this kind of large-scale data retention and distribution possible. Downstream systems can reload and re-process data at will, without impacting the performance of the upstream database that is serving low-latency queries.

Why Avro?

The data extracted from Postgres could be encoded as JSON, or Protobuf, or Thrift, or any number of formats. However, I believe Avro is the best choice. Gwen Shapira has written about the advantages of Avro for schema management, and I’ve got a blog post comparing it to Protobuf and Thrift. The Confluent stream data platform guide gives some more reasons why Avro is good for data integration.

Bottled Water inspects the schema of your database tables, and automatically generates an Avro schema for each table. The schemas are automatically registered with Confluent’s schema registry, and the schema version is embedded in the messages sent to Kafka. This means it “just works” with the stream data platform’s serializers: you can work with the data from Postgres as meaningful application objects and rich datatypes, without writing a lot of tedious parsing code.

The translation of Postgres datatypes into Avro is already fairly comprehensive, covering all the common datatypes, and providing a lossless and sensibly typed conversion. I intend to extend it to support all of Postgres’ built-in datatypes (of which there are many!) — it’s some effort, but it’s worth it, because good schemas for your data are tremendously important.

Inside the bottle factory

The logical decoding output plugin

An interesting property of Postgres’ logical decoding feature is that it does not define a wire format in which change data is sent over the network to a consumer. Instead, it defines an output plugin API, which receives a function call for every insert, update or delete. Bottled Water uses this API to read data in the database’s internal format, and serializes it to Avro.

The output plugin must be written in C using the Postgres extension mechanism, and loaded into the database server as a shared library. This requires superuser privileges and filesystem access on the database server, so it’s not something to be undertaken lightly. I understand that many a database administrator will be scared by the prospect of running custom code inside the database server. Unfortunately, this is the only way logical decoding can currently be used.

Fortunately, you don’t have to install the plugin on your leader database — you can just use a follower (replica, hot standby) for that purpose. That way you can be sure that the plugin cannot corrupt your data, crash the main database instance or affect its performance.

Bottled Water architecture

The client daemon

Besides the plugin (which runs inside the database server), Bottled Water consists of a client program which you can run anywhere. It connects to the Postgres server and to the Kafka brokers, receives the Avro-encoded data from the database, and forwards it to Kafka.

The client is also written in C, because it’s easiest to use the Postgres client libraries that way, and because some code is shared between the plugin and the client. It’s fairly lightweight and doesn’t need to write to disk.

What happens if the client crashes, or gets disconnected from either Postgres or Kafka? No problem. It keeps track of which messages have been published and acknowledged by the Kafka brokers. When the client restarts after an error, it replays all messages that haven’t been acknowledged. Thus, some messages could appear twice in Kafka, but no data should be lost.

Related work

Various other people are working on similar problems:

  • Decoderbufs is an experimental Postgres plugin by Xavier Stevens that decodes the change stream into a Protocol Buffers format. It only provides the logical decoding plugin part of the story — it doesn’t have the consistent snapshot or client parts (Xavier mentions he has written a client which reads from Postgres and writes to Kafka, but it’s not open source).
  • pg_kafka (also from Xavier) is a Kafka producer client in a Postgres function, so you could potentially produce to Kafka from a trigger.
  • PGQ is a Postgres-based queue implementation, and Skytools Londiste (developed at Skype) uses it to provide trigger-based replication. Bucardo is another trigger-based replicator. I get the impression that trigger-based replication is somewhat of a hack, requiring schema changes and fiddly configuration, and incurring significant overhead. Also, none of these projects seems to be endorsed by the PostgreSQL core team, whereas logical decoding is fully supported.
  • Sqoop recently added support for writing to Kafka. To my knowledge, Sqoop can only take full snapshots of a database, and not capture an ongoing stream of changes. Also, I’m unsure about the transactional consistency of its snapshots.
  • For those using MySQL, James Cheng has put together a list of change capture projects that get data from MySQL into Kafka. AFAIK, they all focus on the binlog parsing piece and don’t do the consistent snapshot piece.

Status of Bottled Water

At present, Bottled Water is alpha-quality software. It’s more than a proof of concept — quite a bit of care has gone into its design and implementation — but it hasn’t yet been tested in any real-world scenarios. It’s definitely not ready for production use right now, but with some testing and tweaking it will hopefully become production-ready in future.

We’re releasing it as open source now in the hope of getting feedback from the community. Also, a few people who heard I was working on this have been bugging me to release it :-)

The README has more information on how to get started. Please let us know how you get on! Also, I’ll be talking more about Bottled Water at Berlin Buzzwords in June — hope to see you there.


ApacheCon 2015

$
0
0

I was at ApacheCon 2015 in Austin, Texas a couple of weeks ago. The following is a short summary of some of the trends that I observed at the conference.

There is a lot of usage and interest in Apache Kafka.

I gave a talk on “Introduction to Apache Kafka Tuesday afternoon. About a quarter of the attendees went to the talk and the room was full. We also hosted a Kafka meetup on Tuesday evening for conference attendees. I provided an overview of the Confluent Platform 1.0 release. In particular, I described Schema Registry and how it can be used with Kafka to captured structured feeds of stream data. Most participants seem to understand the value of having schemas and a well defined policy for schema evolution. On Wednesday morning, Todd Palino from LinkedIn gave a well received talk on Kafka at Scale: a multi-tiered architecture, in which he shared lots of valuable experience of running Kafka in production.

Tony Ng from Ebay gave a talk on Pulsar: Realtime Analytics at Scale leveraging Kafka, Hadoop and Kylin. Pulsar is an open source real time analytics and stream processing framework. Its typical usage includes reporting/dashboards, business activity/monitoring, personalization, marketing/advertising, fraud and bot detection. Currently, it’s processing about 0.5 million events/sec at Ebay. Similar to other stream processing systems, Pulsar supports getting data from different channels including Kafka, file, http, etc. It also has a pluggable processing layer. Pulsar supports SQL on stream data through Esper. One key distinguishing feature I found is that Pulsar integrates with Druid and Apache Kylin, a Hadoop based OLAP engine to build cubes.

There is a growing awareness and interest in the space of realtime data ingestion.

Joe Will gave a talk on Apache NiFi: Better Analytics Demands Better DataflowApache Nifi is a new incubator project and was originally developed at the NSA. In short, it is a data flow management system similar to Apache Camel and Flume. It’s mostly intended for getting data from a source to a sync. It can do light weight processing such as enrichment and conversion, but not heavy duty ETL. One unique feature of Nifi is its built-in UI, which makes the management and the monitoring of the data flow convenient. The whole data flow pipeline can be drawn on a panel. The UI shows statistics such as in/out byte rates, failures and latency in each of the edges in the flow. One can pause and resume the flow in real time in the UI. Nifi’s Architecture is also a bit different from Camel and Flume. There is a master node and many slave nodes. The slaves are running the actual data flow and the master is for monitoring the slaves. Each slave has a web server, a flow controller (thread pool) layer, and a storage layer. All events are persisted to a local content repository. It also stores the lineage information in a separate governance repository, which allows it to trace at the event level. Currently, the fault tolerance story in Nifi is a bit weak. The master is a single point of failure. There is also no redundancy across the slaves. So, if a slave dies, the flow stops until the slave is brought up again.

There are lots of activities in container-based resource management.

Christos Kozyrakis gave a talk on Apache Mesos in which he outlined some of the new development in Mesos. One can now do resource allocation based on the framework roles. For example, one can configure to only give SSDs to database services. It seems that there is a Mesos DNS for service discovery. Christos also showed the results of an interesting analysis. Even with the usage of Mesos, in a data center, typical server cpu utilization is only about 30% most of the time. The reason is due to the curse of over-provisioning. Mesos is trying to improve the server utilization by reporting unused resources to the master as “best effort” resources. Those resources can then be used for low priority tasks. If not done carefully, such an approach may have negative performance impact. For example, the low priority task can pollute L3 cache and cause the latency to increase in realtime applications. Mesos tries to address this problem by using isolators at different levels.

In summary, Kafka had great presence in ApacheCon this year. It’s great to see a lot of interest in Kafka and what Confluent is building. If you like working on Kafka and are interested in helping us build realtime stream processing technologies at Confluent, please let us know.

Real-time stream processing: The next step for Apache Flink

$
0
0

This is a guest blog post written by Kostas Tzoumas and Stephan Ewen from data Artisans

Stream processing is becoming very popular with open source projects like Apache Kafka, Apache Samza, Apache Storm, Apache Spark’s Streaming module, as well as cloud services like Google Cloud Dataflow.

In this post, we introduce Apache Flink as a streaming system. Flink fills what we perceive as an open gap in the space of streaming systems, by providing a combination of –

  1. a proper foundation with a low-latency stream processor
  2. expressive APIs that enable programmers to quickly develop streaming data applications,
  3. flexible operator state and streaming windows, and
  4. efficient fault tolerance including streams and state.

Many companies are transitioning their data infrastructure from a batch architecture to a real-time streaming architecture. Static files in distributed file systems like HDFS and relational databases are being augmented with event streams. Workloads that used to be executed with batch processors can be executed with stream processors delivering lower latency applications.

flink blog post pic 1

There are many drivers for this transition. First, many data sets and use cases that formed the bulk of the workloads in Hadoop clusters were always event-based (for example, machine logs). Second, stream processors are maturing to the extent that they can handle more complex workloads. Stream processors can, in principle, execute most of the workloads that batch processors can with lower latency. Thus, there is little reason to use a batch processor when a stream processor can handle the same workload. Finally, new applications such as processing sensor data often require continuous queries, and can only be served by a streaming architecture.

Typically, a stream architecture consists of the following three coarse components:

flink pic 2

  1. A module to gather the event streams from a variety of data sources
  2. A module to collect the streams and make them available for consumption
  3. A module to analyze the streams creating derived streams

The first step, gathering the streams, depends heavily on the primary data sources. Events coming from databases, machine-generated logs, and even sensors need to be cleaned, schema-tized and forwarded to a central place.

The second step, collecting the streams in one central place is the major focus of projects like Apache Kafka. Kafka offers brokers that collect streams, log and buffer them in a fault tolerant manner, as well as distribute them among the various consumers that are interested in the streams.

The third and final step is actually doing some analysis on the streams. This is the step that for example creates counters, aggregates them, correlates values, joins different streams together, and creates derived data streams that can be further integrated into the platform. This step, the data routing and transformation fabric for streaming platforms that provides the necessary delivery guarantees along with the high-level programmability required by data analysts, is the step that Apache Flink implements.

Flink Primer

At a high level, Flink is a framework for distributed data analysis. The core of Flink is a streaming iterative data flow engine. On top of the engine, Flink exposes two language-embedded fluent APIs, the DataSet API for consuming and processing batch data sources and the DataStream API for consuming and processing event streams. Recent additions to the project are domain-specific libraries and APIs built on top of these APIs, such as a Machine Learning library, a graph analysis library (Gelly), and a SQL-like API (Table). In addition, other projects can use Flink as a backend, currentlyGoogle Cloud Dataflow, Apache SAMOA (pending), and Apache MRQL. Flink is also extensible from the backend, being able to run in a variety of environments, such as locally, standalone Flink clusters, embedded in other applications, on top of YARN, and, as of recently, on top of Apache Tez (for batch programs only).

pic3

Flink can be used for both batch and stream processing. The dataflow runtime treats batch programs as a special case of streaming programs. Confused? You shouldn’t be. Think for example about the fact that even MapReduce reads HDFS files as file streams. Flink’s runtime is following this idea consistently to the end, streaming batch data all the way, including shuffles. Batch data sources are to Flink finite streams, while proper event streams are infinite streams. Admittedly, the batch API and ecosystem of Flink is more developed at this point, but the streaming ecosystem is catching up quickly.

In order to explain this better, let us jump into the details. Instead of talking about how Flink processes data streams in isolation, we would like to present 8 building blocks that, in our experience, are fundamental for stream processors and how Flink supports or plans to support these.

Building blocks for stream processing

While building Flink as a stream processor we have identified eight building blocks that we deem essential for classifying a system as a stream processor. We break these features down to four categories, each category containing two features:

  • Basics:
    1. Pipelining
    2. Replay
  • Handling state:
    1. Operator state
    2. State backup and restore
  • Application development:
    1. High-level APIs and languages
    2. Integration with batch sources
  • Large deployments:
    1. High availability
    2. Automatic scale-out and scale-in

Pipelining (#1) means that events can flow through the system freely without unnecessary delays. Operator state (#3) means that, unlike batch processing which is based on immutable transformations on data sets, stream operators need to keep state. Stream replay (#2), and state backup and restore (#4) form the foundation of tolerating machine failures. High-level APIs (#5) are needed for productive application development, as well as being able to combine streams with static sources (#6). A high-availability system (#7) is needed when streaming jobs need to run for long time periods, and elastic resource scaling (#8) is needed to run streaming jobs in shared clusters. Let’s drill into each of these features, and discuss how Flink implements those.

Few years back, Stonebraker et al. had proposed 8 system-level requirements for stream processing. Interestingly, there is a straightforward mapping of how these requirements can be met by a stream processor that implements the above building blocks combined with a fault-tolerant streaming source such as Apache Kafka. See the end of this post for the mapping.

Basics

Pipelining

The first functionality that almost all streaming systems provide is pipelining. Let us look at a simple windowed count where one operator tokenizes lines of an incoming log, and another operator creates a sliding window over grouped data and counts elements within the window.

pic4

Pipelining means that all tasks must be online at the same time and let the data stream through, even if tasks reside on different nodes. For example, the parallel tasks s1, t1, and w2 must all run concurrently, and allow the data to continuously flow through the pipeline even if t1 and w2 reside on different nodes. This requires that network shuffles cannot be performed as in batch processors by letting t1 produce its output before starting w2, but need to be performed in a streaming manner.

Replay

A data streaming system must be able to “replay” parts of a stream, reproducing the results of a user program. Replay is important for several reasons, most notably for recovering from failures, which start to happen frequently when scaling systems to large clusters. In the case of batch operations (finite streams), one can simply store and replay the entire stream and re-execute the entire task. For infinite streams, the system needs to retain a certain history of the stream and coordinate what to retain, for how long, and what to replay.

Basically all systems keep some form of upstream backup and acknowledge (sets of) records. Apache Storm, for example, tracks records with unique IDs. Put simply, operators send messages acknowledging when a record has been processed. Records are dropped from the backup when the have been fully acknowledged.

Flink follows a more coarse-grained approach and acknowledges sequences of records instead of individual records. Periodically, the data sources inject “checkpoint barriers” into the data stream (see figure below). The barriers flow through the data stream, “pushing” records before them and triggering operators to emit all records that depend only on records before the barrier.

pic5

Once all sinks have received the barriers, Flink knows that all records before the barriers will never be needed again. They have been fully processed, including all downstream records that operators create from the original records. Flink then “group acknowledges” all those records together by letting the sources discard the backuped streams to the point where the barriers were injected.

In case of an operator failure, the data flow is reset to the last completed checkpoint (point of barrier injection) and simply resumes from there. Note that this mechanism naturally extends to having upstream backup not only at the sources, but also at intermediate operators. In that case, subsets of the dataflow may be reset in case of a failure.

It is also noteworthy that this mechanism never stops the stream from running while checkpointing, because it only inserts some messages into the stream. It is a very lightweight mechanism and thus supports low checkpoint intervals. It is even possible to have multiple barriers in the stream, meaning that multiple checkpoints are concurrently happening.

Note: At this point, Flink does not back up records itself but tightly cooperates with a persistent data source (like Kafka) to retain the records. This is crucial in any case to make sure that failures do not lose the records that are in-between the data source and the upstream backup buffers. We plan on also adding internal upstream backups to Flink, in order to relieve pressure off the sources and recover the parallel data stream in a more fine-grained manner.

Operator state, state backup and restore

Applications that go beyond simple filters or line-by-line transformations of the input need stateful operators. There are three kinds of state that are offered by Flink:

  • User-defined state: As Flink is a streaming dataflow system, operators are continuously running, and can contain user-defined state in the form of Java/Scala objects. This is the most elementary (but most flexible) operator state, but is not backed up or restored by the system in cases of failures.
  • Managed state: User-defined operations will be able to use a special interface to put their state into an object that is backed up and restored in case of failures. Flink contains managed state internally, and the community is working on exposing this to the user-facing APIs shortly.
  • Windowed streams: Flink offers the ability to create a finite stream from an infinite stream using (sliding) windows over the stream. The contents of those windows is special form of state that is managed by the system.

State backup and restore refers to the ability of the system to reliably recover the state in the case of a failure. In Flink, we are using a variation of the Chandy-Lamport algorithm for consistent asynchronous distributed snapshots. Here, the state backup works hand-in-hand with the checkpoint barriers, which, when reaching an operator, trigger the backup of the operator state. Restoring the state happens together with resetting the stream to the last checkpoint’s barrier. Because stream- and state restore are aligned by that, this mechanism gives exactly-once processing guarantees.

The nice point of this approach is that state checkpointing may proceed asynchronously without ever interfering with and stalling the actual data flow. The data can keep streaming through the system regardless of the frequency of checkpoints.

Easing application development

We have seen that a stream processing system must keep the data moving, be able to replay streams for recovery, as well as provide an abstraction for state and be able to backup and restore that state.

Equally important is the user experience that a stream processing system provides in terms of high-level programmability. Typically, as stream platforms become more mainstream, users do not want to program applications in terms of low-level message-passing interfaces, but would like to have higher-level constructs to define the content of their applications.

Also, given that many companies nowadays do not employ a streaming system in production, most analysis is currently done using batch processors. It is very important for stream processing systems to offer an easy path for users to transition from batch-oriented to stream-oriented analysis, and also incorporate both batch and stream data sources in their analysis.

The approach that Flink takes is to provide two fluent APIs embedded in programming languages (Java and Scala) The DataSet API, whose main data structure is a distributed collection of objects of finite size. The DataStream API, whose main data structure is a potentially unbounded distribute collection of objects. Both data structures are logical, in the sense that they are merely API concepts. Internally, the system generates streaming dataflows from these programs. The APIs look very similar. Here is, for example,WordCount in the (batch) DataSet API:

Screen Shot 2015-05-05 at 9.51.52 PM

and here is a windowed WordCount in the DataStream API with a sliding window of 5 seconds that triggers computation every 1 second:

Screen Shot 2015-05-05 at 9.46.28 PM

Programmers that come from the batch processing world can very easily make the switch to streaming programs. The basic new notions are the one of an unbounded stream and a window (to bound an unbounded stream).

In addition to familiar APIs for developers, Flink also aims to provide libraries and DSLs on top of the DataStream API for data scientists. A first glimpse of this is the Table API, the project’s first step towards SQL support. The Table API supports equally batch and streaming. Thus, people can analyze streaming data with SQL-like syntax, e.g.,

Screen Shot 2015-05-05 at 9.46.36 PM

Large deployments

High availability

We outlined the mechanism to recover from worker failures in the sections above. Compensating for failures of the master is current work in progress. The basic idea is the same as in most other systems: Have multiple masters nodes for failover and use ZooKeeper to elect the leader. The changing state that the Flink master maintains is rather small for running dataflows: It consists mainly of the checkpoint IDs and pointers to the latest backuped state from all operators.

Scale-in and Scale-out

For long running jobs, it is often desirable to change the parallelism of operators without losing the operator state. The way we plan to implement this in Flink is by shutting the dataflow down with a checkpoint, and bringing the dataflow back up with a different parallelism. Think of it as a controlled failure and resume operation. The hardest thing here is re-distributing the state over a different number of workers. A typical approach to that (taken for example in Apache Samza) is to always over-partition the state and have each parallel operator handle multiple partitions. Repartitioning means then simple changing the assignment between partitions and operators.

What about the Lambda and Kappa architectures?

When talking about streaming, the Lambda architecture is an oft-mentioned concept. The Lambda architecture uses a batch processor for the “heavy lifting”, and a stream processor in order to keep up with the fresh data, taking care of the updates that the batch processor did not have time to deal with. While the Lambda architecture philosophy contains several concepts that are extremely useful when building large-scale services, we believe that the specific use of a batch and stream processor in this manner is to some extent an artifact of stream processors not being powerful enough when Lambda was proposed.

This is also the idea behind the Kappa architecture, which advocates using a stream processor for all processing. Flink is exactly this, a stream processor that surfaces both a batch and a streaming API, which are both executed by the underlying stream processor. However, we have observed that there are certain additions needed to the Kappa architecture.

While a streaming dataflow system can certainly execute batch programs (this is what Flink does), the framework should include few different code paths for batch and streaming. In particular, Flink includes an optimizer that is only applicable for batch programs, as query optimization is very different for batch and stream use cases. Second, the behavior of batch and streaming is different for moving intermediate data and state out-of-core. When memory runs out, operators in batch programs should spill to disk. In stream programs, spilling to disk is one option, the others being load shedding and scale-out. Finally, fault tolerance is somewhat easier in batch programs, as finite streams (e.g., static files) can be replayed completely, which enables an optimized code path for batch recovery.

History of stream processing in Flink

The relationship between Flink and stream processing has a somewhat funny history. Interestingly, Flink always had pipelining, the first essential feature for stream processing.

The decision to make Flink a pipelined engine rather than a batch engine (such as Hadoop MapReduce) was made for efficiency reasons. Pipelining engines can be faster in executing batch programs than batch engines in a wide variety of use cases. For example, in typical data warehousing scenarios where a fact table is joined with multiple dimension tables pipelining the fact table results in huge savings. This is the reason that subsequent SQL-on-Hadoop solutions, most notably Impala, are pipelined engines instead of batch engines like Hadoop MapReduce. Flink is today in fact a hybrid engine that can support both batch and pipelined data shuffles natively.

Flink’s DataStream API followed later, when we realized that surfacing the underlying streaming capabilities at an API layer would be interesting for many applications. Flink is architected as a layered system, which means that its APIs are exactly this: APIs. Therefore, adding a streaming API did not require changes to the underlying engine. Both the DataSet and the DataStream APIs create dataflow programs that “look” the same to the Flink engine.

Looking ahead

The Flink community is currently very active in further developing Flink as a stream processor, both by fixing missing features, and by adding new functionality. Our goal is to provide both the necessary foundation with a proper stream processor, the high-level programmability needed for productive application development, the ability to seamlessly intermix batch and streaming in applications, as well as the necessary tools for production deployments.

Appendix

Few years back, Stonebraker et al. published a paper that outlined 8 requirements for stream processing. While these are system-level requirements and not individual features, there is a good mapping between Stonebraker’s requirements and the 8 building blocks we define above:

  1. Keep the data moving: This is achieved by feature #1, and implementing feature #4 without blocking the dataflow.
  2. Query using StreamSQL: This corresponds roughly to feature #5. While a variant of SQL is important, we believe that it is equally important to offer language-embedded APIs.
  3. Handle stream imperfections: This point refers mainly to handling “straggling” records that arrive late, and offering API constructs to handle those; it does not map to any of our requirements so far. Flink does not offer a mechanism to handle straggling records at this point, but may later adopt a similar way of handling this as Google Cloud Dataflow, via user-defined watermarks and straggler-handlers (see also MillWheels).
  4. Generate predictable outcomes: While global ordering of streams is not commonly offered by distributed stream processors, a variant of exactly-once delivery guarantees is commonly achieved by features #2 and #4.
  5. Integrate stored and streaming data: This is achieved by feature #6.
  6. Guarantee data safety and availability: This requirement is achieved by a fault tolerant streaming source (typically Apache Kafka), a highly available master (feature #6), and a fault tolerance mechanism (comprised of features #2 and #4).
  7. Partition and scale applications automatically: this corresponds to feature #8.
  8. Process and Respond Instantaneously: this is a holistic, system-level requirement that is achieved by an efficient implementation of all the above.

Compatibility Testing For Apache Kafka

$
0
0

Testing is one of the hardest parts of building reliable distributed systems. Kafka has long had a set of system tests that cover distributed operation but this is an area that is simply never good enough.

At Confluent and Cloudera we’ve both been working on improving the testing capabilities for Kafka.

An area of particular importance is compatibility. Companies that want to build reliable data real-time data flow and processing around Kafka need to be able to do so without fear of incompatibilities that could arise release to release or between versions of Kafka from different vendors.

We’re announcing today a project with the folks at Cloudera and the rest of the open source community to develop high quality tests to certify API and protocol compatibility between versions and distributions.

We’ll be doing this as part of the normal Apache development process, much as we do any other Kafka development.

We think ensuring this kind of compatibility is one of the key aspects of building a healthy ecosystem of systems, applications, and processing frameworks, that is the core of our stream data platform goal.


How I Learned to Stop Worrying and Love the Schema, part 1

$
0
0

The rise in schema-free and document-oriented databases has led some to question the value and necessity of schemas. Schemas, in particular those following the relational model, can seem too restrictive, and the case has been made that software development can be faster and more agile without them. However, just because it’s possible to go without schemas doesn’t mean it’s wise to do so – this sort of local optimization can cause huge headaches within even a small organization.

The hazards of many languages

Imagine for a moment that you work at a company where all employees are required to speak only in their native language. Intercommunication can work, but either everyone has to be multilingual, or expensive translators must be added for every pair of languages spoken in the company. Even if you have a sophisticated and efficient way of getting messages from place to place, you’re still stuck with the overhead of constant translation.

Furthermore, even if your company phases out, say Latin, unless you are willing to discard all Latin records, you’re either stuck employing your Latin translators for the rest of eternity, or with the work of converting all Latin records into a new language.

Compare this to a company which standardized on a single language from the start. Every single form of communication is easier, and every message can be consumed many times at zero extra cost. Although there is an up-front cost in the sense that new employees must already know the language or be trained in it, the payoff is huge and permanent.

Having no standardized way of defining data across an organization presents a similar problem. It may be fine in the short term, but it quickly causes unnecessary difficulties, and it just doesn’t scale when you imagine multiplying by potentially thousands of different categories of data.

Temp-o-meter – a tale of woe

Let’s illustrate with a simple example. Say that your company has built a smartphone app called temp-o-meter which collects temperature data and sends it back to HQ. Version 0.1 was produced in a hurry, and produces simple comma separated data points with the format

“device_id, temp_celsius, timestamp, latitude, longitude”

A typical data point might look like this:

“123,100,1431811836.081531,37.386052,-122.083851”

Pretty reasonable, but time passes, and the team decides JSON is easier to work with, and temp-o-meter v0.2 produces data like this:

{
    “device_id”: 123,
    “timestamp”:1431811836.081531,
    “temperature”: 212,
    “latitude”: 37.386052,
    “longitude”: -122.083851
}

The problem is, some stage(s) in the downstream pipeline must now have logic to differentiate between CSV and JSON, and this logic must be aware that in the CSV format, temperature readings are in Celsius, but temperature stored in JSON is in Fahrenheit. What’s more, there may be some users who never upgrade their app, so the different versions of this data will continue to be published indefinitely.

Granted, this example is a bit contrived – clearly, for a given type of data, it’s not great to represent it with a mix of formats such as CSV, JSON, or XML, etc. However, just standardizing on a format such as JSON without schemas is not enough. Standardizing on JSON without using schemas is a little like standardizing on the Roman alphabet without standardizing on a language – everyone can easily read and write individual letters, but that still doesn’t guarantee they can read the messages!

Let’s go a little further with the temp-o-meter example and pretend that we live in a science-fiction world where not only phones, but even things like watches can produce streams of data. temp-o-meter needs to be ported, but lucky for the watch team, JSON is now the standard, and the format of temperature data was loosely documented on an obscure wiki page.

Here’s what the temp-o-meter watch team came up with:

{
    “device_id”: “watch_345”,
    “timestamp”: “Tue 05-17-2015 6:00”,
    “temperature”: 212,
    “latitude”: 37.386052,
    “longitude”: -122.083851
}

Not so bad on its own. However, although the format is now JSON, and although the field names are identical, the watch team used slightly different data formats in a few of the fields. “device_id” is no longer parseable as an integer, and “timestamp” is a completely different format.

Why is this a problem? Suppose there is an application which consumes data produced by the temp-o-meter v0.2 – it might have a chunk of code like this:

# Consumer parses a chunk of data from temp-o-meter v0.2
data = json.loads(data_chunk)

device_id = int(data[‘device_id’])
timestamp = float(data[‘timestamp’])
temperature = float(data[‘temperature’])
latitude = float(data[‘latitude’])
longitude = float(data[‘longitude’])

This consumer must be upgraded before the watch team releases, otherwise it will be completely broken when it encounters the (unintentionally) new data format. Despite the fact that the watch team and phone teams now both use JSON, different components of the system are now tightly coupled because the data’s ‘schema’ is embedded in both producers and consumers of this data. The ability to safely and independently evolve different components of the system has been hamstrung.

Had this company been using schemas, the watch and phone teams could have simply reused the same schema, avoiding the need for the watch team to reinvent the wheel, and preventing subtle incompatibilities which ultimately break a bunch of downstream consumers. By sharing the schema between watch and phone apps, this unintentional data evolution would have easily been avoided.

DRY your data definitions with schemas

At this stage in our little story, the ‘definition’ of temp-o-meter’s temperature data is decidedly un-DRY: it is encoded informally in temp-o-meter v0.1, temp-o-meter v0.2, some wiki pages, the watch app, and in all the various consumers which later parse and analyze this data.

On the other hand, by using schemas, the data definition for a particular kind of data exists in a single place. What’s more, schemas serve as self-contained and automatically enforceable contracts between writers and readers of data. Though they don’t remove the need for testing, schemas make testing data compatibility significantly simpler and can nip an entire class of problems in the bud by preventing corrupt, malformed or incompatible data from ever being published in the first place.

For additional compelling reasons to use schemas, it’s worth revisiting this post on Stream Data Platforms.
In the next post on schemas, I’ll talk more about how schemas can provide a powerful tool to help evolve data formats in a sane and compatible way. Stay tuned for part 2!



Using logs to build a solid data infrastructure (or: why dual writes are a bad idea)

$
0
0

This is an edited transcript of a talk I gave at the Craft Conference 2015. The video and slides are also available.

How does your database store data on disk reliably? It uses a log.
How does one database replica synchronise with another replica? It uses a log.
How does a distributed algorithm like Raft achieve consensus? It uses a log.
How does activity data get recorded in a system like Apache Kafka? It uses a log.
How will the data infrastructure of your application remain robust at scale? Guess what…

Logs are everywhere. I’m not talking about plain-text log files (such as syslog or log4j) – I mean an append-only, totally ordered sequence of records. It’s a very simple structure, but it’s also a bit strange at first if you’re used to normal databases. However, once you learn to think in terms of logs, many problems of making large-scale data systems reliable, scalable and maintainable suddenly become much more tractable.

Drawing from the experience of building scalable systems at LinkedIn and other startups, this talk explores why logs are such a fine idea: making it easier to maintain search indexes and caches, making your applications more scalable and more robust in the face of failures, and opening up your data for richer analysis, while avoiding race conditions, inconsistencies and other ugly problems.

Using logs to create a solid data infrastructure

Hello! I’m Martin Kleppmann, and I work on large-scale data systems, especially the kinds of systems that you find at internet companies. I used to work at LinkedIn, contributing to an open source stream processing system called Samza.

In the course of that work, my colleagues and I learnt a thing or two about how to build applications such that they are operationally robust, reliable and perform well. In particular, I got to work with some fine people like Jay Kreps, Chris Riccomini and Neha Narkhede. They figured out a particular architectural style for applications, based on logs, that turns out to work really well. In this talk I will describe that approach, and show how similar patterns arise in various different areas of computing.

What I’m going to talk about today isn’t really new — some people have known about these ideas for a long time. However, they aren’t as widely known as they should be. If you work on a non-trivial application, something with more than just one database, you’ll probably find these ideas very useful.

Designing Data-Intensive Applications

At the moment, I’m taking a sabbatical to write a book for O’Reilly, called “Designing Data-Intensive Applications”. This book is an attempt to collect the important fundamental lessons we’ve learnt about data systems in the last few decades, covering the architecture of databases, caches, indexes, batch processing and stream processing.

The book is not about any one particular database or tool – it’s about the whole range of different tools and algorithms that are used in practice, and their trade-offs, their pros and cons. This talk is partly based on my research for the book, so if you find it interesting, you can find more detail and background in the book. The first seven chapters are currently available in early release.

Stereotypical three-tier architecture

Anyway, let’s get going. Let’s assume that you’re working on a web application. In the simplest case, it probably has the stereotypical three-tier architecture: you have some clients (which may be web browsers, or mobile apps, or both), which make requests to a web application running on your servers. The web application is where your application code or “business logic” lives.

Whenever the application wants to remember something for the future, it stores it in a database. And whenever the application wants to look up something that it stored previously, it queries the database. This approach is simple to understand and works pretty well.

Web app with DB, cache, search, graph index, message queue and workers

However, things usually don’t stay so simple for long. Perhaps you get more users, making more requests, your database gets slow, and you add a cache to speed it up – perhaps memcached or Redis, for example. Perhaps you need to add full-text search to your application, and the basic search facility built into your database is not good enough, so you end up setting a separate indexing service such as Elasticsearch or Solr.

Perhaps you need to do some graph operations that are not efficient on a relational or document database, for example for social features or recommendations, so you add a separate graph index to your system. Perhaps you need to move some expensive operations out of the web request flow, and into an asynchronous background process, so you add a message queue which lets you send jobs to your background workers.

And it gets worse…

Web app with a horrendous mess of storage services

By now, other parts of the system are getting slow again, so you add another cache. More caches always make things faster, right? But now you have a lot of systems and services, so you need to add metrics and monitoring so that you can see whether they are actually working, and the metrics system is another system in its own right.

Next, you want to send notifications, such as email or push notifications to your users, so you chain a notification system off the side of the job queue for background workers, and it perhaps needs some kind of database of its own to keep track of stuff. But now you’re generating a lot of data that needs to be analysed, and you can’t have your business analysts running big expensive queries on your main database, so you add Hadoop or a data warehouse, and load the data from the database into it.

Now that your business analytics are working, you find that your search system is no longer keeping up… but you realise that since you have all the data in HDFS anyway, you could actually build your search indexes in Hadoop and push them out to the search servers, and the system just keeps getting more and more complicated…

…and the result is complete and utter insanity.

Insanity

How did we get into that state? How did we end up with such complexity, where everything is calling everything else, and nobody understands what is going on?

It’s not that any particular decision we made along the way was bad. There is no one database or tool that can do everything that our application requires – we use the best tool for the job, and for an application with a variety of features that implies using a variety of tools.

Also, as a system grows, you need a way of decomposing it into smaller components in order to keep it manageable. That’s what microservices are all about. But if your system becomes a tangled mess of interdependent components, that’s not manageable either.

Same data in different form

Simply having many different storage systems is not a problem in itself: if they were all independent from each other, it wouldn’t be a big deal. The real trouble here is that many of them end up containing the same data, or related data, but in different forms.

For example, the documents in your full-text indexes are typically also stored in a database, because search indexes are not intended to be used as systems of record. The data in your caches is a duplicate of data in some database (perhaps joined with other data, or rendered into HTML, or something) – that’s the definition of a cache.

Also, denormalization is just another form of duplicating data, similar to caching – if some value is too expensive to recompute on reads, you may store that value somewhere, but now you need to also keep it up-to-date when the underlying data changes. Materialized aggregates, such as the count, sum or average of a bunch of records (which you often get in metrics or analytics systems) are again a form of redundant data.

I’m not saying that this duplication of data is bad – far from it. Caching, indexing and other forms of redundant data are often essential for getting good performance on reads. However, keeping the data in sync between all these various different representations and storage systems becomes a real challenge.

Our challenge: data integration

For lack of a better term I’m going to call this the problem of “data integration”. With that I really just mean “making sure that the data ends up in all the right places”. Whenever a piece of data changes in one place, it needs to change correspondingly in all the other places where there is a copy or derivative of that data.

So how do we keep these different data systems in sync? There are a few different techniques.

A popular approach is so-called dual writes:

Dual writes

Dual writes is simple: it’s your application code’s responsibility to update data in all the right places. For example, if a user submits some data to your web app, there’s some code in the web app that first writes the data to your database, then invalidates or refreshes the appropriate cache entries, then re-indexes the document in your full-text search index, and so on. (Or maybe it does those things in parallel – doesn’t matter for our purposes.)

The dual writes approach is popular because it’s easy to build, and it more or less works at first. But I’d like to argue that it’s a really bad idea, because it has some fundamental problems. The first problem is race conditions.

The following diagram shows two clients making dual writes to two datastores. Time flows from left to right, following the black arrows:

Race condition with dual writes

Here, the first client (teal) is setting the key X to be some value A. They first make a request to the first datastore – perhaps that’s the database, for example – and set X=A. The datastore responds saying the write was successful. Then the client makes a request to the second datastore – perhaps that’s the search index – and also sets X=A.

At the same time as this is happening, another client (red) is also active. It wants to write to the same key X, but it wants to set the key to a different value B. The client proceeds in the same way: it first sends a request X=B to the first datastore, and then sends a request X=B to the second datastore.

All these writes are successful. However, look at what value is stored in each database over time:

Race condition with dual writes

In the first datastore, the value is first set to A by the teal client, and then set to B by the red client, so the final value is B.

In the second datastore, the requests arrive in a different order: the value is first set to B, and then set to A, so the final value is A. Now the two datastores are inconsistent with each other, and they will permanently remain inconsistent until sometime later someone comes and overwrites X again.

An the worst thing: you probably won’t even notice that your database and your search indexes have gone out of sync, because no errors occurred. You’ll probably only realize six months later, while you’re doing something completely different, that your database and your indexes don’t match up, and you’ll have no idea how that could have happened.

That alone should be enough to put anyone off dual writes. But wait, there’s more…

Updating denormalized data

Let’s look at denormalized data. Say, for example, you have an application where users can send each other messages or emails, and you have an inbox for each user. When a new message is sent, you want to do two things: add the message to the list of messages in the user’s inbox, and also increment the user’s count of unread messages.

You keep a separate counter because you display it in the user interface all the time, and it would be too slow to query the number of unread messages by scanning over the list of messages every time you need to display the number. However, this counter is denormalized information: it’s derived from the actual messages in the inbox, and whenever the messages change, you also need to update the counter accordingly.

Let’s keep this one simple: one client, one database. Think about what happens over time: first the client inserts the new message into the recipient’s inbox. Then the client makes a requiest to increment the unread counter.

Update of denormalized data fails

However, just in that moment, something goes wrong – perhaps the database goes down, or a process crashes, or the network gets interrupted, or someone unplugs the wrong network cable. Whatever the reason, the update to the unread counter fails.

Now your database is inconsistent: the message has been added to the inbox, but the counter hasn’t been updated. And unless you periodically recompute all your counter values from scratch, or undo the insertion of the message, it will forever remain inconsistent.

Of course, you could argue that this problem was solved decades ago by transactions: atomicity, the “A” in “ACID”, means that if you make several changes within one transaction, they either all happen or none happen. The purpose of atomicity is to solve precisely this issue – if something goes wrong during your writes, you don’t have to worry about a half-finished set of changes making your data inconsistent.

Wrapping two writes in a transaction

The traditional approach of wrapping the two writes in a transaction works fine in databases that support it, but many of the new generation of databases don’t, so you’re on your own.

Also, if the denormalized information is stored in a different database – for example, if you keep your emails in a database but your unread counters in Redis – then you lose the ability to tie the writes together into a single transaction. If one write succeeds, and the other fails, you’re going to have a difficult time clearing up the inconsistency.

Some systems support distributed transactions, based on 2-phase commit for example. However, many datastores nowadays don’t support it, and even if they did, it’s not clear whether distributed transactions are a good idea in the first place. So we have to assume that with dual writes, the application has to deal with partial failure, which is difficult.

Our challenge: data integration

So, back to our original question. How do we make sure that all the data ends up in all the right places? How do we get a copy of the same data to appear in several different storage systems, and keep them all consistently in sync as the data changes?

As we saw, dual writes isn’t the solution, because it can introduce inconsistencies due to race conditions and partial failures. How can we do better?

Stupidly simple solution: totally ordered sequence of records

I’m a fan of stupidly simple solutions. The great thing about simple solutions is that you have a chance of understanding them and convincing yourself that they’re correct. And in this case, the simplest solution I can see is to do all your writes in a fixed order, and to store them in that fixed order.

If you do all your writes sequentially, without any concurrency, then you have removed the potential for race conditions. Moreover, if you write down the order in which you make your writes, it becomes much easier to recover from partial failures, as I will show later.

So, the stupidly simple solution that I propose looks like this: whenever anyone wants to write some data, we append that write to the end of a sequence of records. That sequence is totally ordered, it’s append-only (we never modify existing records, only ever add new records at the end), and it’s persistent (we store it durably on disk).

The picture above shows an example of such a data structure: moving left to right, it records that we first wrote X=5, then we wrote Y=8, then we wrote X=6, and so on.

The ubiquitous log

That data structure has a name: we call it a log.

The interesting thing about logs is that they pop up in many different areas of computing. Although it may seem like a stupidly simple idea that can’t possibly work, it actually turns out to be incredibly powerful.

One line from nginx access log

When I say “logs”, the first thing you probably think of is textual application logs of the style you might get from Log4j or Syslog. For example, the above is one line from an nginx server’s access log, telling me that some IP addresses requested a certain file at a certain time. It also includes the referrer, the user-agent, the response code and a few other things.

Sure, that’s one kind of log, but when I talk about logs here I mean something more general. I mean any kind of data structure of totally ordered records that is append-only and persistent. Any kind of append-only file.

Logs are everywhere: DB storage engines

In the rest of this talk, I’d like to run through a few examples of how logs are used in practice. It turns out that logs are already present in the databases and systems you use every day. And once we understand how logs are used in various different systems, we’ll be in a better position to understand how they can help us solve the problem of data integration.

I’d like to talk about four different places where logs are used, and the first is in the internals of database storage engines.

B-tree example

Do you remember B-Trees from your algorithms classes? They are a very widely used data structure for storage engines – almost all relational databases, and many non-relational databases, use them.

To summarize them briefly: a B-Tree consists of pages, which are fixed-size blocks on disk, typically 4 or 8 kB in size. When you want to look up a particular key, you start with one page, which is at the root of the tree. The page contains pointers to other pages, and each pointer is tagged with a range of keys: for example, if your key is between 0 and 100, you follow the first pointer; if your key is between 100 and 300, you follow the second pointer; and so on.

The pointer takes you to another page, which further breaks down the key range into sub-ranges. And eventually you end up at the page containing the particular key you’re looking for.

Now what happens if you need to insert a new key/value pair into a B-tree? You have to insert it into the page whose key range contains the key you’re inserting. If there is enough spare space in that page, no problem. But if the page is full, it needs to be split into two separate pages.

B-tree node split

When you split a page, you need to write at least three pages to disk: the two pages that are the result of the split, and the parent page (to update the pointers to the split pages). However, these pages may be stored at various different locations on disk.

This raises the question: what happens if the database crashes (or the power goes out, or something else goes wrong) halfway through the operation, after only some of those pages have been written to disk? In that case, you have the old (pre-split) data in some pages, and the new (post-split) data in other pages, and that’s bad news. You’re most likely going to end up with dangling pointers or pages that nobody is pointing to. In other words, you’ve got a corrupted index.

Now, storage engines have been doing this for decades, so how do they make B-trees reliable? The answer is that they use a write-ahead log (WAL).

A write-ahead log is a particular kind of log, i.e. an append-only file on disk. Whenever the storage engine wants to make any kind of change to the B-tree, it must first write the change that it intends to make to the WAL. Only after it has been written to the WAL, and durably written to disk, it is allowed to modify the actual B-tree.

This makes the B-tree reliable: if the database crashes while data was being appended to the WAL, no problem, because the B-tree hasn’t been touched yet. And if it crashes while the B-tree is being modified, no problem, because the WAL contains the information about what changes were about to happen. When the database comes back up after the crash, it can use the WAL to repair the B-tree and get it back into a consistent state.

This is our first example to show that logs are a really neat idea.

Log-structured storage

Now, storage engines didn’t stop with B-trees. Some clever folks realized that if we’re writing everything to a log anyway, we might as well use the log as the primary storage medium. This is known as log-structured storage, which is used in HBase and Cassandra, and a variant appears in Riak.

In log-structured storage we don’t always keep appending to the same file, because it would become too large and it would be too difficult to find the key we’re looking for. Instead, the log is broken into segments, and from time to time the storage engine merges segments and discards duplicate keys. Segments may also be internally sorted by key, which can make it easier to find the key you’re looking for, and also simplifies merging. However, these segments are still logs: they are only written sequentially, and they are immutable once written.

As you can see, logs play an important role in storage engines.

Logs are everywhere: DB replication

Let’s move on to the second example where logs are used: database replication.

Replication is a feature that you find in many databases: it allows you to keep a copy of the same data on several different nodes. That can be useful for spreading the load, and it also means that if one node dies, you can fail over to another one.

Leader-follower replication

There are a few different ways of implementing replication, but a common choice is to designate one node as the leader (also known as primary or master), and the other replicas as followers (also known as standby or slave). I don’t like the master/slave terminology, so I’m going to stick with leader/follower.

Whenever a client wants to write something to the database, it needs to talk to the leader. Read-only clients can use either the leader or the follower (although the follower is typically asynchronous, so it may have slightly out-of-date information if the latest writes haven’t yet been applied).

When clients write data to the leader, how does that data get to the followers? Big surprise: they use a log! They use a replication log, which may in fact be the same as the write-ahead log (this is what Postgres does, for example) or it may be a separate replication log (MySQL does this).

Follower applies writes in order of replication log

The replication log works as follows: whenever some data is written to the leader, it is also appended to the replication log. The followers read that log in the order it was written, and apply each of the writes to their own copy of the data. As a result, each follower processes the same writes in the same order as the leader, and thus it ends up with a copy of the same data.

Even if the writes happen concurrently on the reader, the log still contains the writes in a total order. Thus, the log actually removes the concurrency from the writes – it “squeezes all the non-determinism out of the stream of writes”, and on the follower there’s no doubt about the order in which the writes happened.

So what about the dual-writes race condition we discussed earlier?

Race condition with dual writes

This race condition cannot happen with leader-based replication, because clients don’t write directly to the followers. The only writes processed by followers are the ones they receive from the replication log. And since the log fixes the order of those writes, there is no ambiguity over which one happened first.

Update of denormalized data fails

And what about the second problem with dual writes that we discussed earlier? This could still happen: a follower could successfully process the first write from a transaction, but fail to process the second write from the transaction (perhaps because the disk is full, or the network is interrupted).

Network interruption between leader and follower

If the network between the leader and the follower is interrupted, the replication log cannot flow from the leader to the follower. This could lead to an inconsistent replica, as we discussed previously. How does database replication recover from such errors and avoid becoming inconsistent?

Notice that the log has a very nice property: because the leader only ever appends to it, we can give each record in the log a sequential number that is always increasing (which we might call log position or offset). Furthermore, followers only process it in sequential order (from left to right, i.e. in order of increasing log position), so we can describe a follower’s current state with a single number: the position of the latest record it has processed.

When you know a follower’s current position in the log, you can be sure that all the prior records in the log have already been processed, and none of the subsequent records have been processed.

This is great, because it makes error recovery quite simple. If a follower becomes disconnected from the leader, or it crashes, the follower just needs to store the log position up to which it has processed the replication log. When the follower recovers, it reconnects to the leader, and asks for the replication log starting from the last offset that it previously processed. Thus, the follower can catch up on all the writes that it missed while it was disconnected, without losing any data or receiving duplicates.

The fact that the log is totally ordered makes this recovery much simpler than if you had to keep track of every write individually.

Logs are everywhere: distributed consensus

The third example of logs in practice is in a different area: distributed consensus.

Examples of consensus

Achieving consensus is one of the well-known and often-discussed problems in distributed systems. It is important, but it is also surprisingly difficult to solve.

An example of consensus in the real world would be trying to get a group of friends to agree on where to go for lunch. This is a distinctive feature of a sophisticated civilization, and can be a surprisingly difficult problem, especially if some of your friends are easily distractible (so they don’t always respond to your questions) or if they are fussy eaters.

Closer to our usual domain of computers, an example of where you might want consensus is in a distributed database system: for instance, you may require all your database nodes to agree on which node is the leader for a particular partition (shard) of the database.

It’s pretty important that they all agree on who’s leader: if two different nodes both think they are leader, they may both accept writes from clients. Later, when one of them finds out that it was wrong and it wasn’t leader after all, the writes that it accepted may be lost. This situation is known as split brain, and it can cause nasty data loss.

The Raft consensus protocol

There are a few different algorithms for implementing consensus. Paxos is perhaps the most well-known, but there are also Zab (used by Zookeeper), Raft and others. These algorithms are quite tricky and have some non-obvious subtleties. In this talk, I will just very briefly sketch one part of the Raft algorithm.

In a consensus system, there are a number of nodes (three in this diagram) which are in charge of agreeing what the value of a particular variable should be. A client proposes a value, for example X=8 (which may mean that node X is the leader for partition 8), by sending it to one of the Raft nodes. That node collects votes from the other nodes. If a majority of nodes agree that the value should be X=8, the first node is allowed to commit the value.

When that value is committed, what happens? In Raft, that value is appended to the end of a log. Thus, what Raft is doing is not just getting the nodes to agree on one particular value – it’s actually building up a log of values that have been agreed over time. All Raft nodes are guaranteed to have exactly the same sequence of committed values in their log, and clients can consume this log.

Raft commits a value by appending it to a log

Once the newly agreed value has been committed, appended to the log and replicated to the other nodes, the client that originally proposed the value X=8 is sent a response saying that the system succeeded in reaching consensus, and that the proposed value is now part of the Raft log.

(As a theoretical aside, the problems of consensus and atomic broadcast – that is, creating a log with exactly-once delivery – are reducible to each other. This means Raft’s use of a log is not just a convenient implementation detail, but also
reflects a fundamental property of the consensus problem it is solving.)

Logs are everywhere: Kafka

Ok. We’ve seen that logs really are a recurring theme in surprisingly many areas of computing: storage engines, database replication and consensus. As the fourth and final example, I’d like to talk about Apache Kafka, another system that is built around the idea of logs. The interesting thing about Kafka is that it it doesn’t hide the log from you. Rather than treating the log as an implementation detail, Kafka exposes it to you, so that you can build applications around it.

You may have heard of Kafka before. It’s an open source project that was originally developed at LinkedIn, and is now a lively Apache project with many different contributors and users.

Kafka producers and consumers

The typical use of Kafka is as a message broker (message queue) – so it is somewhat comparable to AMQP, JMS and other messaging systems. Kafka has two types of clients: producers (which send messages to Kafka) and consumers (which subscribe to streams of messages in Kafka).

For example, producers may be your web servers or mobile apps, and the types of messages they send to Kafka might be logging information – e.g. events that indicate which user clicked which link at which point in time. The consumers are various processes that need to find out about stuff that is happening: for example, to generate analytics, to monitor for unusual activity, to generate personalized recommendations for users, and so on.

Kafka architecture: a partitioned log

The thing that makes Kafka interestingly different from other message brokers is that it is structured as a log. In fact, it has many logs! Data streams in Kafka are split into partitions, and each partition is a log (a totally ordered sequence of messages). Different partitions are completely independent from each other, so there is no ordering guarantee across different partitions. This allows different partitions to be handled on different servers, which is important for the scalability of Kafka.

Each partition is stored on disk and replicated across several machines, so it is durable and can tolerate machine failure without data loss. Producing and consuming logs is very similar to what we saw previously in the context of database replication:

  • Every message that is sent to Kafka is appended to the end of a partition. That is the only write operation supported by Kafka: appending to the end of a log. It’s not possible to modify past messages.
  • Within each partition, messages have a monotonically increasing offset (log position). To consume messages from Kafka, a client reads messages sequentially, starting from a particular offset. That offset is managed by the consumer.

Let’s return to the data integration problem from the beginning of this talk.

Web app with a horrendous mess of storage services

Say you have this tangle of different datastores, caches and indexes that need to be kept in sync with each other. Now that we have seen a bunch of examples of practical applications of logs, can we use what we’ve learnt to figure out how to build these systems in a better way?

Stop doing dual writes!

Firstly, we need to stop doing dual writes. As discussed, it’s probably going to make your data inconsistent, unless you have very carefully thought about the potential race conditions and partial failures that can occur in your application.

And note this inconsistency isn’t just a kind of “eventual consistency” that is often quoted in asynchronous systems. What I’m talking about here is permanent inconsistency – if you’ve written two different values to two different datastores, due to a race condition or partial failure, that difference won’t simply resolve itself. You’d have to take explicit actions to search for data mismatches and resolve them (which is difficult, since the data is constantly changing).

We need a better approach than dual writes for keeping different datastores in sync.

Instead, embrace the log

What I propose is this: rather than having the application write directly to the various datastores, the application only appends the data to a log (such as Kafka). All the different representations of this data – your databases, your caches, your indexes – are constructed by consuming the log in sequential order.

Each datastore that needs to be kept in sync is an independent consumer of the log. Every consumer takes the data in the log, one record at a time, and writes it to its own datastore. The log guarantees that the consumers all see the records in the same order; by applying the writes in the same order, the problem of race conditions is gone. This looks very much like the database replication we saw earlier!

And what about the problem of partial failure? What if one of your stores has a problem and can’t accept writes for a while?

Update of denormalized data fails

That problem is also solved by the log: each consumer keeps track of the log position up to which it has processed the log. When the error in the datastore-writing consumer is resolved, it can resume processing records in the log from the last position it previously reached. That way, a datastore won’t lose any updates, even if it’s offline for a while. This is great for decoupling parts of your system: even if there is a problem in one datastore, the rest of the system remains unaffected.

The log, the stupidly simple idea of putting your writes in a total order, strikes again.

Just one problem remains: the consumers of the log all update their datastores asynchronously, so they are eventually consistent. Reading from them is like reading from a database follower: they may be a little behind the latest writes, so you don’t have a guarantee of read-your-writes (and certainly not linearizability).

I think that can be overcome by layering a transaction protocol on top of the log, but that’s a researchy area which so far hasn’t been widely implemented in production systems. For now, a better option is to extract the log from a database:

Using change data capture

This approach is called change data capture, which I wrote about recently (and implemented on PostgreSQL). As long as you’re only writing to a single database (not doing dual writes), and getting the log of writes from the database (in the order in which they were committed to the DB), then this approach works just as well as making your writes to the log directly.

As this database in front of the log applies writes synchronously, you can use it to make reads that require “immediate consistency” (linearizability), and enforce constraints (e.g. requiring that account balances never go negative). Going via a database also means that you don’t need to trust the log as your system of record (which may be a scary prospect if it’s implemented with a new technology) – if you have an existing database that you know and like, and you can extract a change log from that database, you can still get all the advantages of a log-oriented architecture. I’ll be talking more about this topic in an upcoming conference talk.

To close, I’d like to leave you with a thought experiment:

Thought experiment: could you make all your writes through a log?

Most APIs we work with have endpoints for both reading and writing. In RESTful terms, GET is for reading (i.e. side-effect-free operations) and POST, PUT and DELETE are for writing. These endpoints for writing are ok if you only have one system you’re writing to, but if you have more than one such system, you quickly end up with dual writes and all their aforementioned problems.

Imagine a system with an API in which you eliminate all the endpoints for writing. Imagine that you keep all the GET requests, but prohibit any POST, PUT or DELETE. Instead, the only way you can send writes into the system is by appending them to a log, and having the system consume that log. (The log must be outside of the system, so that you can have several consumers for the same log.)

For example, imagine a variant of Elasticsearch in which you cannot write documents through the REST API, but only write documents by sending them to Kafka. Elasticsearch would internally include a Kafka consumer that takes documents and adds them to the index. This would actually simplify some of the internals of Elasticsearch, since it would no longer have to worry about concurrency control, and replication would be simpler to implement. And it would sit neatly alongside other tools that may be consuming the same log.

My favorite feature of this log-oriented architecture is this: if you want to build a new derived datastore, you can just start a new consumer at the beginning of the log, and churn through the history of the log, applying all the writes to your datastore. When you reach the end, you’ve got a new view onto your dataset, and you can keep it up-to-date by simply continuing to consume the log!

This makes it really easy to try out new ways of presenting your existing data, for example to index it another way. You can build experimental new indexes or views onto your data without interfering with any of the existing data. If the result is good, you can shift users to read from the new view; if it isn’t, you can just discard it again. This gives you tremendous freedom to experiment and adapt your application.

Further reading


Confluent will be at QCon NYC next week

$
0
0

Some of us from Confluent will be speaking at QCon NYC next week about Apache Kafka and Confluent’s stream data platform. Here are some things to look forward to from us.

Tutorial: Capturing and processing streaming data with Apache Kafka

Tuesday June 9th 9am-12pm

Kafka provides high throughput, low latency pub/sub messaging and many large companies are quickly adopting it to handle their realtime and streaming data at large scale. But what can you use it for, and how do you get started? Come to Confluent’s tutorial conducted by our first engineer, Ewen Cheslack-Postava on June 9th at 9am to find out.

We’ll start out with an overview of Kafka starting from the basics. You’ll learn about Kafka’s unifying abstraction, a partitioned and replicated low-latency commit log. Then we’ll discuss concrete applications of Kafka across multiple domains so you can see how Kafka can work for your company.

With a solid understanding of Kafka fundamentals, you’ll develop an end-to-end application that  performs anomaly detection on streaming data to see how quickly you can get up and running with Kafka. The implementation will be broken into two parts. First, you’ll take an existing front-end application and instrument it with a Kafka producer to store user activity events in Kafka. Second, you’ll build a distributed, fault tolerant service that detects and reports anomalies in the activity data.

By the end of the session, you’ll understand and be able to apply all the core functionality of Kafka.

And, the fun doesn’t stop there because you can still attend…

The Many Faces of Apache Kafka: Leveraging real-time data at scale

Thursday June 11th 1:40pm-2.30pm

If you are curious about how Kafka is adopted at large scale in production or if you are looking to learn how to adopt Kafka in practice, attend my talk at 1.40pm on June 11th.

Since we open sourced Kafka more than 4 years ago, it has been adopted very widely from web companies like Uber, Netflix, LinkedIn to more traditional enterprises like Cerner, Goldman Sachs and Cisco. These companies use Kafka in a variety of ways – as the infrastructure for ingesting high-volume log data into Hadoop, to collect operational metrics for monitoring and alerting applications, for low latency messaging use cases, and to power near realtime stream processing.

In this talk, you will learn how Kafka’s unique architecture allows it to be used both for real time processing and as a bus for feeding batch systems like Hadoop. You will also learn how Kafka is fundamentally changing the way data flows through an organization and presents new opportunities for processing data in real time that were not possible before. I will discuss how Kafka impacts the way data is integrated across a variety of data sources and systems.

Lastly, you can expect to learn how you can go about adopting Kafka in your company to leverage real-time data at scale in practice.

If you can’t make it to the tutorial or talk, feel free to ping me or Ewen if you’d like to talk about Apache Kafka or Confluent.


The value of Apache Kafka in Big Data ecosystem

$
0
0

This is a repost of a recent article that I wrote for ODBMS.

In the last few years, there has been significant growth in the adoption of Apache Kafka. Current users of Kafka include Uber, Twitter, Netflix, LinkedIn, Yahoo, Cisco, Goldman Sachs, etc. Kafka is a scalable pub/sub system. Users can publish a large number of messages into the system as well as consume those messages through a subscription, in real time. This blog explains why Kafka is becoming popular and its role in the Big Data eco system.

Limitations of the one-size-fits-all model

For a long time, databases have been the primary place where people store and process the most interesting data. Database vendors keep adding new features such as search, streaming and analytics so that more interesting work can be done inside the database. Overtime, this model is no longer ideal for two reasons. First of all, databases become expensive as people try to collect other data sets such as user behavior tracking records, operational metrics, application logs, etc. Those data sets are equally important as transactional data for deriving new insights, but can be two to three orders of magnitude larger. Since traditional databases typically rely upon expensive high end storage systems (e.g. SAN), storing all those data sets in a database becomes prohibitively expensive. Second, as more features are accumulated, databases become more complicated and it gets harder to add new features while still maintaining all the legacy ones. A multi-year release cycle is common among database vendors.

Emergence of specialized distributed systems

To overcome these limitations, people started building specialized systems in the last 10 years. Those systems were designed to do just one thing, but do it really well. Because of their simplicity, it’s more feasible to build them as distributed systems that run on commodity hardware. As a result, those specialized systems are much more cost effective than SAN-based databases. Often, such systems were built as open source projects, which further drives down the cost of ownership. Also, since those specialized systems focus on just one thing, they can be developed and improved much faster than monolithic databases. Hadoop pioneered this approach. It specializes in offline processing by providing a distributed file system (HDFS) and a computation engine (MapReduce) for storing and processing data in batches. By using HDFS, companies can now afford to collect additional data sets that are valuable, but are too expensive to store in databases. By using MapReduce, people can generate reports and perform analytics on those new data sets in a more cost effective way. This pattern has since been repeated in many other areas.

  • Key/value stores: Cassandra, MongoDB, HBase, etc.
  • Search: Elastic search, Solr, etc.
  • Stream processing: Storm, Spark streaming, Samza, etc.
  • Graph: GraphLab, FlockDB, etc.
  • Time series: Open TSDB, etc.

Such specialized systems enable companies to derive new insights and build new applications that were not possible before.

Feeding specialized systems

While those specialized systems have revolutionized the IT stack, it brings a new challenge: how to feed data into those systems. First, remember there is a wide variety of interesting data types ranging from transactional records, to user tracking data, operational metrics, service logs, etc. Often, the same data set needs to be fed into multiple specialized systems. For example, while application logs are useful for offline log analysis, it’s equally important to search individual log entries. This makes it infeasible to build a separate pipeline to collect each type of data and directly feed it into each relevant specialized system. Second, while Hadoop typically holds a copy of all types of data, it is impractical to feed all other systems off Hadoop since many of them require data more real time than what Hadoop can provide. This is where Kafka comes into play. Kafka has the following nice features.

  • It’s designed as a distributed system and can store high volume of data on commodity hardware.
  • It’s designed as a multi-subscription system. The same published data set can be consumed multiple times.
  • It persists data to disks and can deliver messages to both realtime and batch consumers at the same time without performance degradation.
  • It has built-in redundancy and therefore can be used to provide the reliability needed for mission critical data.

Most of those companies mentioned at the beginning invariably have adopted several of those specialized systems. They use Kafka as a central place to ingest all types of data in real time. The same data in Kafka is then fed to different specialized systems. We refer to this architecture as a stream data platform as depicted in the figure below. Adding additional specialized systems into this architecture is easy since the new system can get its data by simply making an extra subscription to Kafka. stream_data_platform What’s next

The trend in the industry is that multiple specialized systems will co-exist in the Big Data eco system. A stream data platform powered by distributed pub/sub systems like Kafka will play an increasingly important role in this eco system as more companies are moving towards more realtime processing. An impact of this is that one may have one rethink the data curation process. Currently, much of the data curation such as schematizing the data and evolving the schemas is deferred until after the data is loaded into Hadoop. This is not ideal in the stream data platform since the same data curation process then has to be repeated in other specialized systems as well. A better approach is to reason about data duration early when the data is ingested into Kafka. This is part of what we are doing at Confluent and you can find more details in our web site.


How to Write a User Defined Function (UDF) for KSQL

$
0
0

In August 2017, Confluent announced and open sourced KSQL, an open source streaming SQL project for Apache Kafka. While released as developer preview, many developers already use KSQL to build streaming applications on top of Apache Kafka. Use cases include streaming ETL, anomaly detection, and real time dashboards. Check the video from Taking KSQL for a Spin using Real-Time Device Data to see the potential of KSQL for Internet of Things scenarios.

A key component of any SQL language is its functions, and KSQL already has many functions built in. This blog post explains how to write user-defined functions (UDFs) to extend the available set of functions in your KSQL code.

KSQL: the Streaming SQL Engine for Apache Kafka

KSQL is implemented on top of Kafka’s Streams API, and it leverages all its benefits to build scalable, mission-critical streaming applications. A key difference of KSQL to other technologies in this space is that KSQL does not require any coding in Java, Scala, or another programming language. You can write SQL-like queries and deploy them for continuous stream processing

KSQL example query:

Developers and data engineers can opt to either (1) write, execute, and deploy SQL queries on the KSQL servers, or (2) embed the statements into own applications or microservices—no matter if Java, any other programming language like Go, Python or .NET C#, or (3) integrate with KSQL’s REST interface.

 

The difference from KSQL to traditional SQL engines—where you execute a request-response query to a relational database like Oracle or MySQL—is that you define and run continuous queries or “streaming queries.” These queries run forever (or until you terminate them explicitly) to analyze incoming data-in-motion, instead of historical data-at-rest that is stored in, for example, a traditional database.

Built-in KSQL Functions to Process Streaming Data

Functions are one of the key components in SQL. Functions in SQL engines are like functions, operations, or methods in programming languages such as Java, Python, or Go. They accept parameters, perform an action—such as a complex calculation—and return the result of that action as a value. Functions are used within a SQL query to filter, transform, or aggregate data. Many functions are already built into KSQL, and their number increases with every release:

But what do you do if you miss a specific function in the KSQL syntax? Let’s us show you how easy it is to write your own user-defined function.

Before We Begin

In this blog post we explain how to write a UDF for KSQL by adding the UDF code to the KSQL source.  While this approach is still quite straightforward for many users, we will provide a much simpler approach in a future release to implement such UDFs where you will no longer need to modify the KSQL source. In the meantime, please follow the instructions in this blog post if you need to implement your own UDFs right now.

User-Defined Functions in KSQL

In contrast to built-in functions, UDFs are functions provided by the user of a program or environment. A UDF provides a mechanism for extending the functionality of the KSQL engine by adding a function that can be evaluated in standard query language statement.

Using a UDF in KSQL looks just like using a built-in functions in KSQL. Both are registered to the KSQL engine before startup. They expect input parameters and return output values. Here is an example of the built-in function ‘STRINGTOTIMESTAMP’ to convert a string value in the given format into the BIGINT value representing the timestamp:

This function takes the timestamp 2017-12-18 11:12:13.111 (data type: String) and converts it to 1513591933111 (data type: BigInt).

Now let’s take a look at the steps to build your own UDF the same way as the ‘STRINGTOTIMESTAMP’ function above.

How to Create, Deploy, and Test KSQL UDFs

This section shows the steps to build, deploy and test a new UDF. The following example shows how to build simple UDF where you need no additional external Java classes or JAR library. We just write Java code within the UDF to do a conversion of a timestamp from String to BigInt. A more complex UDF might require additional external dependencies to be added to your project.

Step 0: Clone the KSQL project from GitHub

First clone (or fork) the KSQL project with the ‘git clone’ command from https://github.com/confluentinc/ksql. The main sub-project in KSQL for adding UDFs is ‘ksql-engine’ and the package ‘io.confluent.ksql.function’ (https://github.com/confluentinc/ksql/tree/0.4/ksql-engine).

Step 1 (optional): Embed external code or libraries into projectDifferent steps might be needed depending on what your new UDF requires to perform its task:

  • Dependencies on another project such as DeepLearning4J (e.g. the implementation of a complex mathematical function)
  • New classes (e.g. a generated analytic model from machine learning framework)
  • Additional libraries or other dependencies like zip or xml files

You can either simply add new resources like another Java Class to the project or extending the existing Maven POM (ksql-engine/pom.xml).

Step 2: Implement a New UDF Class

This class needs to implement the ‘Kudf’ interface. It includes the core functionality of your new UDF. In this example, we take a look at the class StringToTimeStamp.java:

As you can see, the full implementation is just a few lines of Java code. In general, you need to implement the logic between receiving input and returning output of the UDF in the evaluate() method. You also need to implement exception handling (e.g. invalid input arguments) where applicable. The init() method is empty in this case, but could initialise any required object instances.

Note that this UDF has state: dateFormat can be null or already initialized. However, no worries. You do not have to manage the scope as Kafka Streams (and therefore KSQL) threads are independent of each other. So this won’t cause any issues.

Step 3: Register UDF to FunctionRegistry

The new UDF needs to be registered in the FunctionRegistry class like all the built-in KSQL functions:

The first argument of the KsqlFunction is the return type. The second is the list of types for the UDF arguments. The third is the name of the function. The last is the Java class that implements the UDF functionality.

In our case, the return value is the timestamp as as type “BigInt.” The input parameters are the column name with the timestamp String and the format of the timestamp String. You will be able to call the UDF in a KSQL query the following way: STRINGTOTIMESTAMP(col1, 'yyyy-MM-dd HH:mm:ss.SSS').

That’s it. We just implemented a new UDF and registered it to the KSQL engine. No more coded needed.

 Step 4: (Re-)Build KSQL Project with Maven

Now you need to re-run the Maven build of your KSQL project so that you can use the new UDF.  If you’re in a hurry and don’t mind taking some risks, you can remove the -DskipTests=true -Dcheckstyle.skip=true for a faster build.

 Step 5: Start Kafka Broker and KSQL CLI

In this case, we start a new Kafka broker including Zookeeper via Confluent CLI and then start KSQL CLI:

You can point KSQL at a specific Kafka cluster using command-line parameters

Step 6: Test your new UDF

After starting the Kafka cluster and KSQL engine, we need to prepare the demo. In my case, I create an input topic and generate new test data via command line or continuous script.

     a) Create Kafka Topic

     b) Create Test Data

    c) Create KSQL Stream

   d) Use UDF to Convert the String to Timestamp

   e) Send Test Data to KSQL Stream

As soon as you send the message, KSQL processes it and converts the timestamp. You can see the result in the select query you started in step d): 123 | 1513591933111

For more advanced testing, you should write a script which generates continuous feeds with different test data.

  f) Consume results from any Kafka Client

Since KSQL runs on normal Kafka topics, you can also process and forward any KSQL related messages with the normal Kafka Consumer in any Kafka Clients like Java, .NET, Go or Python or directly consume via Kafka Connect.

Step 7: Debugging and Error Fixing

You should also test the UDF with values which are expected to fail. If you pass the wrong number of arguments to the STRINGTOTIMESTAMP function, you will see an error in the CLI as defined in our implementation:

StringToTimestamp udf should have two input arguments

If your UDF is not working as expected in your KSQL engine (e.g. returning ‘null’), then the best way to find errors and exceptions is to check the KSQL log file (by default generated in a temp folder):

For example, if you add a wrong timestamp format, you will see a similar error message in the log file (as we defined this in the UDF implementation):

Exception running StringToTimestamp( ‘2017-12-18 11:12:13.111’ , yyyy-MM-dd HH:mm:ss.SSS) : Unparseable date: ” ‘2017-12-18 11-12-13-111′”

 Step 8: Use your new UDF in your KSQL Projects

That’s all you need to do to build, deploy and test new UDFs for KSQL. The above steps described manual testing of your new UDF. Of course, you can leverage any tools or frameworks to automate testing and integrate CI/CD tools like Jenkins, as this is just plain Java code and shell scripts. Because KSQL is just a simple Java API, you have the entire Java build and test ecosystem at your disposal.

Build and Share your KSQL UDF

You have learned how easy it is to create and add new User Defined Functions to KSQL. Just a few steps to implement, register and run a new UDF.

In a future KSQL release will make it even easier to build UDFs by removing the need to rebuild the project to add your own UDFs, e.g. by providing a CREATE FUNCTION statement. We encourage you to share your feedback on how you’d like to build your own UDFs.

Where to go from here

If you have enjoyed this article, you might want to continue with the following resources to learn more about KSQL:

If you are interested in contributing to KSQL, we encourage you to get involved by sharing your feedback via the KSQL issue tracker, voting on existing issues by giving your +1, or opening pull requests. Use the #ksql channel in our public Confluent Slack community to ask questions, discuss use cases or help fellow KSQL users.

The post How to Write a User Defined Function (UDF) for KSQL appeared first on Confluent.

Confluent and Apache Kafka in 2017

$
0
0

2017 was an amazing year for Apache Kafka® as an open source project and for Confluent as a company. I wanted to take moment to thank all the contributors and users of the open source project, as well as our customers and partners and the people here at Confluent for contributing to such an outstanding year.

One of the most impressive things for me to see in 2017 was the degree to which companies were starting to view Kafka as a strategic data platform, and the scope and centrality of the applications they were developing. We got to work with companies reimagining retail, healthcare, cars, travel, banking, and virtually every other industry you can imagine, around streams of real-time events. I could not be more excited about this: our vision for Kafka is for it to grow into a central nervous system of the modern company, and we seem to be well on our way to that vision.

We’re already working on a new and exciting set of challenges for 2018, but I thought it would be fun to celebrate a couple of the big steps we took last year. Here are few.

Confluent Milestones

  • We grew our subscription bookings by over 4x year over year with 98 percent customer satisfaction. And yes this is on top of the amazing growth from last year.
  • Confluent more than doubled in size, adding over one hundred new employees to the company. We also added several new members of our leadership team, including Simon Hayes, Vice President of Corporate and Business Development; Sarah Sproehnle, Vice President of Customer Success; and Ivan Ernest, Chief People Officer.
  • We released Confluent Cloud, an Apache Kafka as a Service offering. This offering lets us manage, monitor, upgrade and expand Kafka and Confluent Platform for you as an elastic cloud offering. So, rather than build Kafka-expertise in-house you can get Kafka at the click of a button.
  • We released KSQL, an open source streaming SQL engine that enables continuous, interactive SQL queries on Apache Kafka.
  • We dramatically expanded the capabilities of Confluent Enterprise, our enterprise distribution of Kafka that lets you manage, monitor and scale Kafka in your own datacenter.

Apache Kafka Community Milestones

  • The community participated in two Kafka Summits in San Francisco and New York City with over 1,300 attendees and 30 sponsors across the events. There will be two Kafka Summits in 2018: one in San Francisco and the first-ever international event in London.
  • Apache Kafka is growing into a spectacular open source project. We had more than 19,500 meetup members and tens of thousands of production users. Development velocity continues to accelerate with over 75 new core design proposals (KIPs) completed this year.
  • The community announced the highly-anticipated release of exactly-once semantics, a set of features enabling idempotence and transaction to ensure correct reading, writing and processing for continual data streams. This was a huge effort involving design and coding across the larger Kafka development community.
  • Kafka finally went 1.0, a huge milestone for the whole community.

We’ve come a long way since the earliest Kafka prototypes we did at LinkedIn, but in so many ways the most exciting things lay ahead of us. Stay tuned for what’s next.

The post Confluent and Apache Kafka in 2017 appeared first on Confluent.

KSQL in Action: Real-Time Streaming ETL from Oracle Transactional Data

$
0
0

In this post I’m going to show what streaming ETL looks like in practice. We’re replacing batch extracts with event streams, and batch transformation with in-flight transformation. But first, a trip back through time…

My first job from university was building a data warehouse for a retailer in the UK. Back then, it was writing COBOL jobs to load tables in DB2. We waited for all the shops to close and do their end of day system processing, and send their data back to the central mainframe. From there it was checked and loaded, and then reports generated on it. This was nearly twenty years ago as my greying beard will attest—and not a lot has changed in the large majority of reporting and analytics systems since then. COBOL is maybe less common, but what has remained constant is the batch-driven nature of processing. Sometimes batches are run more frequently, and get given fancy names like intra-day ETL or even micro-batching. But batch processing it is, and as such latency is built into our reporting by design. When we opt for batch processing we voluntarily inject delays into the availability of data to our end users, and to applications in our business that could be driven by this data in real time.

Back in 2016 Neha Narkhede wrote that ETL Is Dead, Long Live Streams, and since then we’ve seen more and more companies moving to adopt Apache Kafka as the backbone of their architectures. With Kafka’s Connect and Streams APIs, as well as KSQL, we have the tools available to make Streaming ETL a reality.

Streaming ETL with Confluent Platform

By streaming events from the source system as they are created, using Kafka’s Connect API, data is available for driving applications throughout the business in real time. Independently and in parallel, that same data can be transformed and processed and be made available to end users as soon as they want it. The key is that we are no longer batch-driven; we are event-driven.

Data enriched through the transform process is streamed back into Kafka. From here it can also be used by other applications. So we can refine raw inbound data, and use the resulting enriched and cleansed data for multiple purposes. Think of all the data cleansing and business logic that gets done as part of ETL…isn’t the resulting data useful in more places than just a static data store?

As a data engineer for an online store, you’re tasked with providing a real-time view for your sales operations team on current website activity. Which important customers are using the site? What’s the rolling value of orders being placed? But as well as an analytical ‘cockpit’ view, we can use the same enriched data to feed an event-driven microservice responsible for notifying the inside sales team when particularly important customers log on to the site. We can utilise the same logic and definitions once, for driving both the analytics and the microservice.

So in this post I’m going to show an example of what streaming ETL looks like in practice. I’m replacing batch extracts with event streams, and batch transformation with in-flight transformation of these event streams. We’ll take a stream of data from a transactional system built on Oracle, transform it, and stream it into Elasticsearch to land the results to, but your choice of datastore is up to you—with Kafka’s Connect API you can stream the data to almost anywhere! Using KSQL we’ll see how to filter streams of events in real-time from a database, how to join between events from two database tables, and how to create rolling aggregates on this data.

Let’s Get Started!

My source system is Oracle 12c, with the Order Entry schema and a transactional workload driven by a tool called Swingbench. I’m not using Swingbench here in its common capacity of load/stress-testing, but instead simply to generate a stream of transactions without needing to have access to a real data feed. To stream the data from Oracle, I’m using Oracle GoldenGate for Big Data. This is one of several Change-Data-Capture (CDC) tools available (others include DBVisit’s Replicate) which all work on the principal of taking the transaction log and streaming events from it to Kafka. There are plenty of other CDC tools around for other databases, including the popular Debezium project which is open-source and currently supports both MySQL and Postgres.

You can see details on the components I’ve used, and how to exactly reproduce them for your own experiments here.

The starting point for this is an inbound stream of events in Kafka from our source system (Oracle, via CDC). This is the “Extract” of our ETL, and is running in real time, event-by-event.

We’re going to apply some transformations to these events, and do so in real time—not batch! We’ll take a small set of the source tables containing:

  • Orders
  • Logon events
  • Customer details

From these we will use KSQL to provide a real time stream of:

  • Customers logging onto the 
application, with a second version of the stream filtered just to show customers of highest value who are logging in
  • Aggregated order counts and values

We’ll also see in a subsequent post how we’ll also use this enriched data that’s being written back to Kafka to drive a microservice. This microservice will send an alert to the inside sales team whenever a long-standing business customer logs on to the site.

Join and Filter Streams of Data from Oracle in KSQL

To join the customers to the logon event data, we will create a Table in KSQL on the Customers topic. We’re making a table because we only want to look at the current state of each customer; if we wanted to see a history of how a customer had changed over time, then we’d want a stream. The Logon data is a sequence of events, and so we just create a Stream on it. Having created both, we then join the two.

Firing up KSQL, first we define the customer table source topic:

ksql> CREATE STREAM CUST_SRC WITH (KAFKA_TOPIC='ora-ogg-SOE-CUSTOMERS-avro', 
VALUE_FORMAT='AVRO');

Message  
----------------  
Stream created  
----------------

Note that we’ve not had to specify the schema of the data, because it’s in Avro format and KSQL pulls the schema automagically from Confluent Schema Registry. To learn more about the schema registry and its importance in building applications see this great presentation from Gwen Shapira hereTo specify the Schema Registry location you need to either pass --schema-registry-url to ksql-cli in local mode, or specify ksql.schema.registry.url in a properties file that you pass as an argument when starting up a standalone KSQL server.

ksql> DESCRIBE CUST_SRC;

 Field             | Type
-----------------------------------------------
 ROWTIME           | BIGINT           (system)
 ROWKEY            | VARCHAR(STRING)  (system)
 OP_TYPE           | VARCHAR(STRING)
 OP_TS             | VARCHAR(STRING)
 CURRENT_TS        | VARCHAR(STRING)
 POS               | VARCHAR(STRING)
 CUSTOMER_ID       | BIGINT
 CUST_FIRST_NAME   | VARCHAR(STRING)
 CUST_LAST_NAME    | VARCHAR(STRING)
 NLS_LANGUAGE      | VARCHAR(STRING)
 NLS_TERRITORY     | VARCHAR(STRING)
 CREDIT_LIMIT      | DOUBLE
 CUST_EMAIL        | VARCHAR(STRING)
 ACCOUNT_MGR_ID    | BIGINT
 CUSTOMER_SINCE    | VARCHAR(STRING)
 CUSTOMER_CLASS    | VARCHAR(STRING)
 SUGGESTIONS       | VARCHAR(STRING)
 DOB               | VARCHAR(STRING)
 MAILSHOT          | VARCHAR(STRING)
 PARTNER_MAILSHOT  | VARCHAR(STRING)
 PREFERRED_ADDRESS | BIGINT
 PREFERRED_CARD    | BIGINT
-----------------------------------------------

We’ll set the topic offset to earliest so that any queries and derived streams that we create contain all of the data to date:

ksql> SET 'auto.offset.reset' = 'earliest';

Let’s have a quick peek at the data:

ksql> SELECT OP_TYPE,OP_TS,CUSTOMER_ID, CUST_FIRST_NAME, CUST_LAST_NAME FROM CUST_SRC LIMIT 1;
I | 2017-09-13 14:50:51.000000 | 74999 | lee | murray
LIMIT reached for the partition.
Query terminated
ksql>

Since we’re going to be joining on the customer ID, we need to rekey the table. KSQL’s DESCRIBE EXTENDED command can be used to inspect details about an object including key:

ksql> DESCRIBE EXTENDED CUST_SRC;

Type                 : STREAM
Key field            :
Timestamp field      : Not set - using <ROWTIME>
Key format           : STRING
Value format         : AVRO
[...]

We can use KSQL to easily rekey a topic, using the PARTITION BY clause:

ksql> CREATE STREAM CUST_REKEYED AS SELECT * FROM CUST_SRC PARTITION BY CUSTOMER_ID;

Check out the key for the new STREAM:

ksql> DESCRIBE EXTENDED CUST_REKEYED;

Type                 : STREAM
Key field            : CUSTOMER_ID
Timestamp field      : Not set - using <ROWTIME>
Key format           : STRING
Value format         : AVRO
Kafka output topic   : CUST_REKEYED (partitions: 4, replication: 1)
[...]

Two things of interest here – the key column is now CUSTOMER_ID, but we can also see that there is a Kafka output topicCUST_REKEYED. Just for completeness, let’s check the key on the Kafka topics, using the awesome kafkacat:

Source topic:

Robin@asgard02 > kafkacat -C -c2 -K: -b localhost:9092 -o beginning -f 'Key:    %k\n' -t ora-ogg-SOE-CUSTOMERS-avro  
Key:    74999_lee_murray_RC_New Mexico_6000.00_lee.murray@ntlworld.com_561_2009-06-05 00:00:00_Business_Electronics_1974-02-27 00:00:00_Y_N_49851_49851  
Key:    75000_larry_perez_VX_Lithuania_6000.00_larry.perez@googlemail.com_514_2011-04-01 00:00:00_Occasional_Health_1960-11-30 00:00:00_Y_Y_114470_114470

Re-keyed topic

Robin@asgard02 > kafkacat -C -c2 -K: -b localhost:9092 -o beginning -f 'Key:    %k\n' -t CUST_REKEYED  
Key:    75000  
Key:    75004

So we’ve got our rekeyed topic. Now let’s build a table on top of it:

ksql> CREATE TABLE CUSTOMERS WITH 
(KAFKA_TOPIC='CUST_REKEYED', VALUE_FORMAT='AVRO', KEY='CUSTOMER_ID');

Message  
---------------  
Table created  
---------------

And finally, query the table that we’ve built:

ksql> SELECT ROWKEY, CUSTOMER_ID, CUST_FIRST_NAME, CUST_LAST_NAME, CUSTOMER_CLASS 
FROM CUSTOMERS LIMIT 5;  
75000 | 75000 | larry | perez | Occasional  
74999 | 74999 | lee | murray | Business  
75004 | 75004 | derrick | fernandez | Prime  
75007 | 75007 | tony | simmons | Occasional  
75014 | 75014 | roy | reed | Business  
LIMIT reached for the partition.  
Query terminated  
ksql>

So – that’s our Customers reference table built and available for querying. Now to bring in the Logon events stream:

ksql> CREATE STREAM LOGON WITH 
(KAFKA_TOPIC='ora-ogg-SOE-LOGON-avro', VALUE_FORMAT='AVRO');

Message  
----------------  
Stream created  
----------------

That was easy! Let’s check we’re getting data from it:

ksql> SELECT LOGON_ID,CUSTOMER_ID,LOGON_DATE FROM LOGON LIMIT 5;  
178724 | 31809 | 2000-11-08 23:08:51  
178725 | 91808 | 2009-06-29 02:38:11  
178726 | 78742 | 2007-11-06 15:29:38  
178727 | 4565 | 2010-03-25 09:31:44  
178728 | 20575 | 2000-05-31 00:22:00  
LIMIT reached for the partition.  
Query terminated

Now for the magic bit…joining the table and stream!

ksql> SELECT L.LOGON_ID, C.CUSTOMER_ID, C.CUST_FIRST_NAME, C.CUST_LAST_NAME, 
C.CUSTOMER_SINCE, C.CUSTOMER_CLASS 
FROM LOGON L LEFT OUTER JOIN CUSTOMERS C ON L.CUSTOMER_ID = C.CUSTOMER_ID;

You may see some nulls in the results, which is as a result of not all logon events having a corresponding customer entry. You can filter these out using:

ksql> SELECT L.LOGON_ID, C.CUSTOMER_ID, C.CUST_FIRST_NAME, C.CUST_LAST_NAME, 
C.CUSTOMER_SINCE, C.CUSTOMER_CLASS 
FROM LOGON L LEFT OUTER JOIN CUSTOMERS C ON L.CUSTOMER_ID = C.CUSTOMER_ID 
WHERE C.CUSTOMER_ID IS NOT NULL LIMIT 5;  
178771 | 75515 | earl | butler | 2002-07-19 00:00:00 | Occasional  
178819 | 76851 | cesar | mckinney | 2000-10-07 00:00:00 | Regular  
178832 | 77941 | randall | tucker | 2010-04-23 00:00:00 | Prime  
178841 | 80769 | ramon | hart | 2011-01-24 00:00:00 | Occasional  
178870 | 77064 | willard | curtis | 2009-05-26 00:00:00 | Occasional  
LIMIT reached for the partition.  
Query terminated

Watch out for this current issue if you’re joining on non-identical datatypes.

Having tested the simple join, we can start to build on it, adding in column concatenation (first + last name)

CONCAT(C.CUST_FIRST_NAME ,CONCAT(' ',C.CUST_LAST_NAME)) AS CUST_FULL_NAME

as well as calculations, here taking date on which the account was opened and using it to determine to the nearest year how long the person has been a customer. The functions used here are

  • STRINGTOTIMESTAMP which converts the string timestamp into an epoch
  • CAST(…AS DOUBLE) so that the BIGINT values can be accurately used in calculations

The completed statement, wrapped as a CREATE STREAM AS SELECT (CSAS) so that it can be used as the basis of subsequent queries, as well as instantiated as an underlying Kafka topic that can be used outside of KSQL, is as follows:

ksql> CREATE STREAM LOGON_ENRICHED AS 
SELECT L.LOGON_ID, L.LOGON_DATE, C.CUSTOMER_ID, 
CONCAT(C.CUST_FIRST_NAME ,CONCAT(' ',C.CUST_LAST_NAME)) AS CUST_FULL_NAME, 
C.CUST_FIRST_NAME, C.CUST_LAST_NAME, C.CUSTOMER_SINCE, 
C.CUSTOMER_CLASS, C.CUST_EMAIL, 
(CAST(C.ROWTIME AS DOUBLE)-CAST(STRINGTOTIMESTAMP(CUSTOMER_SINCE,'yyyy-MM-dd HH:mm:ss') 
AS DOUBLE))/ 60 / 60 / 24 / 1000/365 AS CUSTOMER_SINCE_YRS 
FROM LOGON L 
LEFT OUTER JOIN CUSTOMERS C 
ON L.CUSTOMER_ID = C.CUSTOMER_ID ;

From the derived stream, we can then start querying both the original and derived columns, with a nice clear and understandable query:

ksql> SELECT LOGON_ID, LOGON_DATE, CUST_FULL_NAME, CUSTOMER_CLASS, CUSTOMER_SINCE_YRS FROM LOGON_ENRICHED;  
178726 | 2007-11-06 15:29:38 | lloyd black | Occasional | 10.771086248255962  
178732 | 2009-05-21 06:34:42 | donald hernandez | Occasional | 17.77108626258879  
178742 | 2002-11-26 12:48:03 | kyle owens | Occasional | 15.776565717751144  
178738 | 2004-09-27 05:36:23 | allen griffin | Business | 16.773825992548197  
[...]

We can also start to apply filters to this, either ad-hoc:

ksql> SELECT LOGON_ID, LOGON_DATE, CUST_FULL_NAME, CUSTOMER_CLASS, CUSTOMER_SINCE_YRS 
FROM LOGON_ENRICHED WHERE CUSTOMER_CLASS = 'Prime' LIMIT 5;  
181362 | 2011-02-16 13:01:16 | isaac wong | Prime | 10.771086241850583  
181551 | 2007-01-15 11:21:19 | ryan turner | Prime | 6.762867074898529  
181576 | 2009-07-04 02:19:35 | peter campbell | Prime | 14.779305415810505  
181597 | 2006-07-12 04:54:40 | andres fletcher | Prime | 13.782045160768645  
181631 | 2002-09-08 03:06:16 | john johnson | Prime | 6.762867062690258  
LIMIT reached for the partition.  
Query terminated

or creating a further derived stream:

ksql> CREATE STREAM IMPORTANT_CUSTOMER_LOGONS AS SELECT LOGON_ID, LOGON_DATE, CUST_FULL_NAME, 
CUSTOMER_CLASS, CUSTOMER_SINCE_YRS 
FROM LOGON_ENRICHED 
WHERE CUSTOMER_CLASS = 'Business' AND CUSTOMER_SINCE_YRS > 10;

Message  
----------------------------  
Stream created and running

ksql> SELECT * FROM IMPORTANT_CUSTOMER_LOGONS LIMIT 5;  
1507286630384 | 83960 | 178738 | 2004-09-27 05:36:23 | allen griffin | Business | 16.773825992548197  
1507286630386 | 92074 | 178773 | 2010-02-21 20:04:52 | gabriel garza | Business | 14.779305462899543  
1507286630477 | 76111 | 181737 | 2007-05-17 23:59:36 | ray alvarez | Business | 12.765606788305432  
1507286630401 | 87118 | 178936 | 2006-02-07 22:34:47 | kelly oliver | Business | 17.771086274733637

An important point here is that these derived streams are executing in real time, on events as they arrive, and populating Kafka topics with their results. So whilst the LOGON_ENRICHED stream might be for streaming into a general analytics platform, the IMPORTANT_CUSTOMER_LOGONS stream maybe directly drives a customer operations dashboard or application.

Building Streaming Aggregates in KSQL

As well as denormalising data in order to make analysis easier by making relevant data available in one place, we can use KSQL to aggregate data. By aggregating inbound streams of events we can make available to other applications a real time stream of summary metrics about the events being processed. Aggregations are also a common pattern used in data warehousing to improve the performance of accessing data. Instead of storing data at its base granularity, it is “rolled up” to a higher grain at which it is commonly queried. For example, orders are placed as stream of events, but commonly a business operations analyst will want to know the value of orders placed per hour. Here’s a simple example of calculating just that.

First we define our inbound event stream, which is coming from the ORDERS table on Oracle, streamed into the Kafka topic through the CDC process.

ksql> CREATE STREAM ORDERS_SRC WITH (KAFKA_TOPIC='ora-ogg-SOE-ORDERS-avro', VALUE_FORMAT='AVRO');

Since we’re going to be doing some time-based processing, we need to make sure that KSQL is using the appropriate timestamp value. By default it will use the timestamp of the Kafka message itself, which is the time at which the record was streamed into Kafka from the CDC source. You can see which column is being used with DESCRIBE EXTENDED:

ksql> DESCRIBE EXTENDED ORDERS_SRC;

Type                 : STREAM
Key field            :
Timestamp field      : Not set - using <ROWTIME>
Key format           : STRING
Value format         : AVRO
[...]

You can see the actual timestamp too using the ROWTIME implicit column in any KSQL stream object:

ksql> SELECT TIMESTAMPTOSTRING(ROWTIME, 'yyyy-MM-dd HH:mm:ss') , ORDER_ID, ORDER_DATE 
FROM ORDERS_SRC LIMIT 5;
2017-10-25 10:36:12 | 71490 | 2009-01-05 23:00:00.000000000
2017-10-25 10:36:12 | 71491 | 2011-07-26 01:00:00.000000000
2017-10-25 10:36:12 | 71492 | 2008-04-23 15:00:00.000000000
2017-10-25 10:36:12 | 71493 | 2009-04-03 09:00:00.000000000
2017-10-25 10:36:12 | 71494 | 2009-06-22 23:00:00.000000000
LIMIT reached for the partition.
Query terminated
ksql>

In the above output we can see that the ROWTIME (first column) is completely different from ORDER_DATE (third column). The former is the process timeand the latter is the event time. For the purposes of our aggregations, since we are reporting on a business event (and not the physical processing detail) we want to make sure KSQL uses the event time (ORDER_DATE). Let’s first confirm what format the ORDER_DATE is in:

ksql> SELECT ORDER_DATE FROM ORDERS_SRC LIMIT 5;
2009-01-05 23:00:00.000000000
2011-07-26 01:00:00.000000000
2008-04-23 15:00:00.000000000
2009-04-03 09:00:00.000000000
2009-06-22 23:00:00.000000000
LIMIT reached for the partition.
Query terminated

With this knowledge, we can cast the string column to a timestamp, using STRINGTOTIMESTAMP and the Java time format:

ksql> SELECT ORDER_DATE, STRINGTOTIMESTAMP(ORDER_DATE,'yyyy-MM-dd HH:mm:ss.SSSSSSSSS') 
FROM ORDERS_SRC LIMIT 5;
2009-01-05 23:00:00.000000000 | 1231196400000
2011-07-26 01:00:00.000000000 | 1311638400000
2008-04-23 15:00:00.000000000 | 1208959200000
2009-04-03 09:00:00.000000000 | 1238745600000
2009-06-22 23:00:00.000000000 | 1245708000000
LIMIT reached for the partition.
Query terminated
ksql>

From experience, I can suggest it’s always good to validate that you’ve got the date format strings correct, by checking the epoch value independently, using an online service or simple bash (removing the milliseconds first):

Robin@asgard02 > date -r 1231196400
Mon  5 Jan 2009 23:00:00 GMT

If the epoch doesn’t match the string input, check against the Java time format reference, and pay attention to the case particularly. DD means day of the year whilst dd is day of the month, and MM is the month of the year whilst mm is minutes of the hour. What could possibly go wrong…

Now we have the event time in epoch format, we can use this as the basis for defining an intemediary derived stream from this source one. We’re also going to capture the original ROWTIME since this is useful to know as well (what time the message hit Kafka from the CDC source):

ksql> CREATE STREAM ORDERS_INT_01 AS SELECT ROWTIME AS EXTRACT_TS, ORDER_DATE, 
STRINGTOTIMESTAMP(ORDER_DATE,'yyyy-MM-dd HH:mm:ss.SSSSSSSSS') AS ORDER_DATE_EPOCH, 
ORDER_ID, ORDER_STATUS, ORDER_TOTAL FROM ORDERS_SRC;

 Message
----------------------------
 Stream created and running

ksql> SELECT ORDER_ID, ORDER_TOTAL, TIMESTAMPTOSTRING(EXTRACT_TS,'yyyy-MM-dd HH:mm:ss'), TIMESTAMPTOSTRING(ORDER_DATE_EPOCH, 'yyyy-MM-dd HH:mm:ss') FROM ORDERS_INT_01 LIMIT 5;
71491 | 5141.0 | 2017-10-25 10:36:12 | 2011-07-26 01:00:00
71494 | 3867.0 | 2017-10-25 10:36:12 | 2009-06-22 23:00:00
71498 | 5511.0 | 2017-10-25 10:36:12 | 2007-10-18 05:00:00
71501 | 4705.0 | 2017-10-25 10:36:12 | 2007-08-24 17:00:00
71504 | 6249.0 | 2017-10-25 10:36:12 | 2009-12-01 04:00:00
LIMIT reached for the partition.
Query terminated

The final step is to use the new epoch column as the basis for our new Orders stream, in which we will use the TIMESTAMP property assignment to instruct KSQL to use the event time (ORDER_DATE_EPOCH) as the timestamp for the stream:

ksql> CREATE STREAM ORDERS WITH (TIMESTAMP ='ORDER_DATE_EPOCH') AS 
SELECT EXTRACT_TS, ORDER_DATE_EPOCH, ORDER_ID, ORDER_DATE, ORDER_STATUS, ORDER_TOTAL 
FROM ORDERS_INT_01;

 Message
----------------------------
 Stream created and running

Inspecting DESCRIBE EXTENDED for the new stream shows that the Timestamp field is indeed being driven from the Order date (i.e. event time), and not the time at which the event hit our system:

ksql> DESCRIBE EXTENDED ORDERS;

Type                 : STREAM
Key field            :
Timestamp field      : ORDER_DATE_EPOCH
Key format           : STRING
Value format         : AVRO
Kafka output topic   : ORDERS (partitions: 4, replication: 1)
[...]

Now when we query this stream, and include ROWTIME (which is the actual time value KSQL will use for the aggregation) we can see that it matches what we had in the source ORDER_DATE column – the actual event time:

ksql> SELECT TIMESTAMPTOSTRING(ROWTIME, 'yyyy-MM-dd HH:mm:ss') , ORDER_DATE, ORDER_ID, ORDER_TOTAL 
FROM ORDERS LIMIT 5;
2011-07-26 01:00:00 | 2011-07-26 01:00:00.000000000 | 71491 | 5141.0
2009-06-22 23:00:00 | 2009-06-22 23:00:00.000000000 | 71494 | 3867.0
2008-04-23 15:00:00 | 2008-04-23 15:00:00.000000000 | 71492 | 4735.0
2007-10-18 05:00:00 | 2007-10-18 05:00:00.000000000 | 71498 | 5511.0
2007-08-24 17:00:00 | 2007-08-24 17:00:00.000000000 | 71501 | 4705.0
LIMIT reached for the partition.
Query terminated

Phew! Now to actually build our aggregate:

ksql> CREATE TABLE ORDERS_AGG_HOURLY AS 
SELECT ORDER_STATUS, COUNT(*) AS ORDER_COUNT, MAX(ORDER_TOTAL) AS MAX_ORDER_TOTAL, 
MIN(ORDER_TOTAL) AS MIN_ORDER_TOTAL, SUM(ORDER_TOTAL) AS SUM_ORDER_TOTAL, 
SUM(ORDER_TOTAL)/COUNT(*) AS AVG_ORDER_TOTAL 
FROM ORDERS WINDOW TUMBLING (SIZE 1 HOUR) GROUP BY ORDER_STATUS;

 Message
---------------------------
 Table created and running

This creates a table in KSQL, backed by a Kafka topic. Implicit in the table’s columns is the window time (ROWTIME) and the key (ROWKEY) which is a composite of the GROUP BY columns plus the window. Using ROWTIME formatted in a human-readable format we can inspect the aggregate:

ksql> SELECT TIMESTAMPTOSTRING(ROWTIME, 'yyyy-MM-dd HH:mm:ss') , ROWKEY,  ORDER_STATUS, 
MAX_ORDER_TOTAL, MIN_ORDER_TOTAL, SUM_ORDER_TOTAL, ORDER_COUNT, AVG_ORDER_TOTAL 
FROM ORDERS_AGG_HOURLY LIMIT 5;
2008-04-21 16:00:00 | 4 : Window{start=1208790000000 end=-} | 4 | 4067.0 | 4067.0 | 4067.0 | 1 | 4067.0
2007-11-20 21:00:00 | 4 : Window{start=1195592400000 end=-} | 4 | 3745.0 | 3745.0 | 3745.0 | 1 | 3745.0
2008-08-24 06:00:00 | 7 : Window{start=1219554000000 end=-} | 7 | 7354.0 | 7354.0 | 7354.0 | 1 | 7354.0
2008-03-25 05:00:00 | 3 : Window{start=1206421200000 end=-} | 3 | 2269.0 | 2269.0 | 2269.0 | 1 | 2269.0
2009-11-13 23:00:00 | 3 : Window{start=1258153200000 end=-} | 3 | 2865.0 | 2865.0 | 2865.0 | 1 | 2865.0
LIMIT reached for the partition.
Query terminated

This implicit metadata can be exposed properly with a CTAS:

ksql> CREATE TABLE ORDERS_AGG_HOURLY_WITH_WINDOW AS 
SELECT TIMESTAMPTOSTRING(ROWTIME, 'yyyy-MM-dd HH:mm:ss') AS WINDOW_START_TS, ROWKEY,  
ORDER_STATUS, MAX_ORDER_TOTAL, MIN_ORDER_TOTAL, SUM_ORDER_TOTAL, ORDER_COUNT, AVG_ORDER_TOTAL 
FROM ORDERS_AGG_HOURLY;

Checking out the raw Kafka messages shows that our aggregates are in place along with the window timestamp:

Robin@asgard02 > kafka-avro-console-consumer 
                 --bootstrap-server localhost:9092 
                 --property schema.registry.url=http://localhost:8081 
                 --from-beginning 
                 --topic ORDERS_AGG_HOURLY_WITH_WINDOW 
                 --max-messages 1|jq '.'
{
  "WINDOW_START_TS": "2009-04-10 23:00:00",
  "ORDER_STATUS": 4,
  "MAX_ORDER_TOTAL": 3753,
  "MIN_ORDER_TOTAL": 3753,
  "SUM_ORDER_TOTAL": 33777,
  "ORDER_COUNT": 9,
  "AVG_ORDER_TOTAL": 3753
}
Processed a total of 1 messages

Note that as an aggregate is updated (either by data arriving within the current window, or late-arriving data) it is re-emitted, but with the same key (which includes the window) as before. This means that downstream we just need to take the key as the basis for storing the aggregate, and overwrite an existing keys with new values.

Streaming enriched data from Kafka into Elasticsearch

Let’s now take the data that originated in Oracle, streamed in through Kafka, enriched in KSQL, and land it to Elasticsearch. We can do this using Kafka Connect. We’re going to use a single connector to land the contents of both the enriched logons and the aggregated order metrics into Elasticsearch at once. You can load them individually too if you want.

We’re going to use a Single Message Transform (SMT) to set the Timestamp datatype for LOGON_DATE string. This came from GoldenGate as a string, and in order for Elasticsearch to work seamlessly through Kibana we want the Kafka Connect sink to pass the datatype as a timestamp—which using the SMT will enable. The alternative is to use document templates in Elasticsearch to set the datatypes of certain columns, but SMT are neater in this case. We’ll use an SMT for the WINDOW_START_TS too, as this column we cast as a string for display purposes.

Here is the necessary Kafka Connect configuration to stream the Kafka data from two of the topics populated by KSQL into Elasticsearch:

cat > ~/es_sink.json<<EOF
{
  "name": "es_sink",
  "config": {
    "connector.class": "io.confluent.connect.elasticsearch.ElasticsearchSinkConnector",
    "value.converter": "io.confluent.connect.avro.AvroConverter",
    "value.converter.schema.registry.url": "http://localhost:8081/",
    "key.converter": "org.apache.kafka.connect.storage.StringConverter",
    "key.converter.schemas.enable": false,
    "type.name": "type.name=kafkaconnect",
    "topics": "LOGON_ENRICHED,ORDERS_AGG_HOURLY_WITH_WINDOW",
    "topic.index.map": "LOGON_ENRICHED:logon_enriched,ORDERS_AGG_HOURLY_WITH_WINDOW:orders_agg_hourly",
    "connection.url": "http://localhost:9200",
    "transforms": "convert_logon_date,convert_window_ts",
    "transforms.convert_logon_date.type": "org.apache.kafka.connect.transforms.TimestampConverter$Value",
    "transforms.convert_logon_date.target.type": "Timestamp",
    "transforms.convert_logon_date.field": "LOGON_DATE",
    "transforms.convert_logon_date.format": "yyyy-MM-dd HH:mm:ss",
    "transforms.convert_window_ts.type": "org.apache.kafka.connect.transforms.TimestampConverter$Value",
    "transforms.convert_window_ts.target.type": "Timestamp",
    "transforms.convert_window_ts.field": "WINDOW_START_TS",
    "transforms.convert_window_ts.format": "yyyy-MM-dd HH:mm:ss"
  }
}
EOF

Load connector:

$ confluent load es_sink_logon_enriched -d ~/es_sink_logon_enriched.json

Confirm it’s running:

$ confluent status connectors|  jq '.[]'|  xargs -I{connector} confluent status {connector}|  jq -c -M '[.name,.connector.state,.tasks[].state]|join(":|:")'|  column -s : -t|  sed 's/\"//g'|  sort  
es_sink_logon_enriched_01  |  RUNNING  |  RUNNING

If there’s an error then use confluent log connect to see details.

Verify that the SMT has done the trick for the date column, by inspecting the mapping defined for logons:

$ curl -s "http://localhost:9200/logon_enriched/_mappings"|jq '.logon_enriched.mappings."type.name=kafkaconnect".properties.LOGON_DATE.type'  
"date"

and for the aggregated orders:

$ curl -s "http://localhost:9200/orders_agg_hourly/_mappings"|jq '."orders_agg_hourly".mappings."type.name=kafkaconnect".properties.WINDOW_START_TS.type'  
"date"

Sample the data:

$ curl -s -Xget "http://localhost:9200/logon_enriched/_search?q=rick"|jq '.hits.hits[1]'
{
  "_index": "logon_enriched",
  "_type": "type.name=kafkaconnect",
  "_id": "83280",
  "_score": 7.5321684,
  "_source": {
    "CUST_LAST_NAME": "hansen",
    "CUSTOMER_SINCE_YRS": 11.806762565068492,
    "CUSTOMER_SINCE": "2006-12-16 00:00:00",
    "CUSTOMER_CLASS": "Occasional",
    "LOGON_DATE": "2003-07-03 05:53:03",
    "C_CUSTOMER_ID": 83280,
    "CUST_FULL_NAME": "rick hansen",
    "CUST_EMAIL": "rick.hansen@googlemail.com",
    "LOGON_ID": 65112,
    "CUST_FIRST_NAME": "rick"
  }
}

Add the index in Kibana and now we can monitor in real time what’s happening – using data from Oracle, streamed through Kafka, dynamically enriched and joined, and streamed into Elasticsearch.

Here’s a list of all logins, with the full details of the customer included:

Now filtering all real time logins to show just those of “Business” type:

A useful point here is that whilst we can dynamically filter the data in the end-user tool, we could as easily generate a dedicated stream of justCUSTOMER_CLASS = 'Business' records using KSQL. It comes down to whether the data is to support exploratory/ad-hoc analytics, or to drive a business process that only needs data matching certain criteria.

With our data streaming from the transactional RDBMS system through Kafka and into a datastore such as Elasticsearch, it’s easy to build full dashboards too. These give a real time view over business events as they occur:

With the aggregate landed in our datastore, we can easily view the raw aggregate data in a table:

Building on the aggregates that we have created, we can add to the dashboard we created above, including information about the orders placed:

All of this data driven in real time from our source transaction system! Using Kafka we have been able to stream and persist the raw events, transformed and enriched them with KSQL, and streamed to target datastores such as Elasticsearch with Kafka Connect.

ETL Is Dead, Long Live Streams

We’ve seen in this article how we can stream database changes in real-time into Kafka, and use these to drive multiple applications. With KSQL we can easily transform data, from simple filtering of streams of events from a database, to enriching events from multiple sources, denormalising normalised structures, and creating rolling aggregates. Since KSQL writes transformed streams back to Kafka, we can use predicates in KSQL to easily implement exception detection, driving real-time applications without the need for complex coding. By defaulting to being event-driven, we can build systems that provide data for analytics when it’s needed, and use the same enriched data for driving applications in real-time.

Where To Go From Here

If you have enjoyed this article, you might want to continue with the following resources to learn more about KSQL and Streaming ETL:

If you are interested in contributing to KSQL, we encourage you to get involved by sharing your feedback via the KSQL issue tracker, voting on existing issues by giving your +1, or opening pull requests. Use the #ksql channel in our public Confluent Slack community to ask questions, discuss use cases or help fellow KSQL users.

The post KSQL in Action: Real-Time Streaming ETL from Oracle Transactional Data appeared first on Confluent.

Secure Stream Processing with Apache Kafka, Confluent Platform and KSQL

$
0
0

In this blog post, we first look at stream processing examples using KSQL that show how companies are using Apache Kafka to grow their business and to analyze data in real time. These use cases establish KSQL as part of critical business workflows, which means that securing KSQL is also critical. Then we look at how to secure KSQL and the entire Confluent Platform with encryption, authentication, and authorization.

KSQL Use Cases

Kafka enables companies to transform their business with event-driven architectures. They can deploy real-time data pipelines that organize all of an enterprise’s data around a single source of truth, and then use stream processing to enable new business opportunities and new methods of real-time analysis and decision-making. KSQL is the streaming SQL engine for Apache Kafka that makes it very easy to read, write, and process streaming data in real time, at scale, using a SQL-like syntax. There’s no need to write any code in a programming language like Java or Scala.

Below is a KSQL expression that analyzes customer behavior in a hypothetical financial services business. It automatically qualifies residential mortgage clients who have good credit scores and captures those clients in a stream called qualified_clients. This produces a list of customers that a sales representative could reach out to to offer new (and hopefully valuable!) services.

CREATE STREAM qualified_clients AS
SELECT client_id, city, action
FROM mortgages m
LEFT JOIN clients c ON m.client_id = c.user_id
WHERE c.type = 'Residential'AND c.credit_score > 750;

Companies that have a massive amount of data to process, such as those in the Internet of Things (IoT) domain, rely on Kafka for real-time stream processing, operations, and alerting. Below is another KSQL expression that identifies devices reporting a saturation level more than three times in less than five seconds and creates events in a table called possible_malfunction. Then system can decide to send alerts on those events.

CREATE TABLE possible_malfunction AS
 SELECT device_id, count(*)
 FROM saturation_events
 WINDOW TUMBLING (SIZE 5 SECONDS)
 GROUP BY device_id
 HAVING count(*) > 3

The KSQL examples above demonstrate how important stream processing is to companies across multiple domains—and security is critical to businesses like the ones we’ve illustrated. Financial services need to protect sensitive customer data and IoT companies need to protect their deployments from security vulnerabilities to ensure their operations continue to function and are not compromised by malicious code at the edge.

Securing KSQL

To connect to a secured Kafka cluster, Kafka client applications need to provide their security credentials. In the same way, we configure KSQL such that the KSQL servers are authenticated and authorized, and data communication is encrypted when communicating with the Kafka cluster. We can configure KSQL for:

Here is a sample configuration for a KSQL server:

Save your KSQL configuration parameters in a text file, and then pass this properties file to KSQL as a command line argument when you start it:

$ ksql-server-start ksql.properties

Once you have started KSQL, you can review settings for the configuration parameters:

ksql> SHOW PROPERTIES;

Additionally, KSQL natively supports data in the Avro format through integration with Confluent Schema Registry. If you are using KSQL with HTTPS to Confluent Schema Registry, you need to set the ksql.schema.registry.url to HTTPS, as shown in the properties file above. Additionally, set the KSQL environment variable KSQL_OPTS to define the credentials to use when communicating with the Confluent Schema Registry:

# Define KSQL security credentials when communicating with the Confluent Schema Registry via HTTPS
$ export KSQL_OPTS="-Djavax.net.ssl.trustStore=/etc/kafka/secrets/kafka.client.truststore.jks 
                    -Djavax.net.ssl.trustStorePassword=confluent 
                    -Djavax.net.ssl.keyStore=/etc/kafka/secrets/kafka.client.keystore.jks 
                    -Djavax.net.ssl.keyStorePassword=confluent"

Securing Confluent Platform

Security is not a bolt-on feature—it’s a mindset and a process. With Confluent, your organization can deploy an enterprise solution that supports your organization’s process of greater security awareness. Every single component in the Confluent Platform has been designed with end-to-end security in mind:

from your clients…

  • KSQL
  • Kafka Streams API
  • Confluent Python, Go, .NET, C/C++, JMS clients

…to your production infrastructure…

  • Kafka brokers
  • Kafka Connect
  • Confluent REST proxy
  • Confluent Schema Registry
  • ZooKeeper

…to your monitoring infrastructure…

  • Confluent Control Center

Check out the Confluent Platform demo to see security features working together across the platform. The use case in the demo is a Kafka streaming ETL deployment on live edits to real Wikipedia pages:

CP Demo

You can bring up this demo in less than five minutes, with security enabled on the Kafka cluster, and observe the components working end-to-end:

  • KSQL
  • Kafka Connect
  • Confluent Replicator
  • Confluent Schema Registry
  • Kafka command line tools
  • Confluent Control Center, including Confluent Metrics Reporter and Confluent Monitoring Interceptors

The demo also comes with a playbook of scenarios to get a technical understanding of how Kafka works. The playbook has a security section with an accompanying video series to go through some basic scenarios. For example, you can investigate what happens if a user tries to connect to a Kafka cluster to list topics without providing the proper authentication credentials:

Error: Executing consumer group command failed due to Request METADATA failed on brokers List(kafka1:9091 (id: -1 rack: null))

Or investigate how a broker logs an unauthorized client trying to consume messages from a topic in a Kafka cluster, and then configure ACLs to fix it:

INFO Principal = User:CN=client,OU=TEST,O=CONFLUENT,L=PaloAlto,ST=Ca,C=US is Denied Operation = Describe from host = 172.23.0.7 on resource = Group:test (kafka.authorizer.logger)

The demo also includes security for Confluent Control Center, which is the management and monitoring solution for Kafka. The same business security requirements that apply to the underlying infrastructure (e.g, producer and consumer clients, KSQL servers, etc.) also apply to monitoring infrastructure. This is because the data itself, as well as topic names, consumer group names, etc., may all include sensitive information. Access to it should be protected and must not be compromised.

 

Control Center has a unified configuration experience in supporting the same encryption, authentication, and authorization feature sets as your production Kafka cluster. In contrast, if you are using third-party monitoring tools, it is painful to configure and maintain security in a totally different interface from your production Kafka deployment, if those monitoring tools even support security features at all.

Security Resources

To make configuring Kafka security easier, we have recently improved our security configuration documentation. We have also added a security tutorial as a step-by-step guide for bringing up a secure cluster with Kafka brokers, ZooKeeper, Kafka Connect, Confluent Replicator, and optional monitoring via Confluent Control Center.

The tutorial walks through one example configuration and is great starting point for learning how to configure security on the Confluent Platform. The tutorial takes you through these steps:

  1. creating SSL keys and certificates
  2. configuring SSL encryption and SASL authentication on brokers
  3. configuring clients with the right credentials and security settings
  4. using kafka-acls for authorization
  5. troubleshooting commands

Using this as a reference, you can then customize your deployment per your own requirements:

  • The tutorial authenticates with SASL/PLAIN but if your organization uses Active Directory with Kerberos, you can similarly configure SASL/GSSAPI authentication.
  • The tutorial configures super users and uses Kafka’s out-of-the-box SimpleAclAuthorizer to add ACLs for other users. But the Authorizer interface is extensible, and you can plug in your own authorizer implementation.
  • The tutorial configures a mix of SSL and SASL_SSL listeners on the broker, but you can also add an unsecured PLAINTEXT listener if you are migrating your cluster.
  • You can enable security plugins for Confluent Schema Registry or Confluent REST Proxy to propagate authenticated principals to the Kafka cluster.

Confluent Summary

Companies are transforming their businesses with event-driven architectures. Confluent Platform provides end-to-end security for the streaming platform that enables those architectures to be the central nervous system for their companies.

Take the next steps to see how all the Confluent Platform components work securely together:

Interested in Learning More?

If you’d like to learn more about the Confluent Platform, here are some resources for you:

The post Secure Stream Processing with Apache Kafka, Confluent Platform and KSQL appeared first on Confluent.


KSQL February Release: Streaming SQL for Apache Kafka®

$
0
0

We are pleased to announce the release of KSQL v0.5, aka the February 2018 release of KSQL. This release is focused on bug fixes and as well as performance and stability improvements.

Our focus for KSQL has been on improving its operational experience and on addressing issues and bugs reported by the KSQL user community or surfaced through KSQL’s rigorous testing strategies. With the general availability announcement today, we’re excited to see KSQL in production soon. Our efforts have centered around soak testing with eye towards supporting continuous workloads and operation, bounds testing, and performance testing, much of which runs on a daily basis through automated test suites. For example, our soak testing cluster has racked up over 1,000 hours and runs KSQL workloads 24×7. The performance tests we conduct allow us to understand performance characteristics of stateless and stateful KSQL queries. We currently run over 42 different tests that collect more than 700 metrics.

As they say, April showers bring May flowers. In this case, a relentless February focus on quality and stability will bring a trustworthy, production-ready product when KSQL sees its GA release next month—a future poised for a regular cadence of new features that help you take your KSQL-based systems in the direction you want to go.

Where to go from here

If you have enjoyed this article, you might want to continue with the following resources to learn more about KSQL:

If you are interested in contributing to KSQL, we encourage you to get involved by sharing your feedback via the KSQL issue tracker, voting on existing issues by giving your +1, or opening pull requests. Use the #ksql channel in our public Confluent Slack community to ask questions, discuss use cases or help fellow KSQL users.

The post KSQL February Release: Streaming SQL for Apache Kafka® appeared first on Confluent.

Event Sourcing Using Apache Kafka

$
0
0

Adam Warski is one of the co-founders of SoftwareMill, where he codes mainly using Scala and other interesting technologies. He is involved in open-source projects, such as sttp, MacWire, Quicklens, ElasticMQ and others. He has been a speaker at major conferences, such as JavaOne, LambdaConf, Devoxx and ScalaDays.

 

A Western Jackdaw. In Polish, it is called a Kawka—pronounced /ˈkaf.ka/.

When building an event sourced system, there’s a couple of options available when it comes to persistence. You could use a traditional event-sourcing system like EventStore, a scalable tabular database like Cassandra, or even a good old-fashioned relational database. But it’s also straightforward to do event sourcing on top of Apache Kafka. Let’s see how.

What is event sourcing?

There’s a number of great introductory articles, so this is going to be a very brief introduction. With event sourcing, instead of storing the “current” state of the entities that are used in our system, we store a stream of events that relate to these entities. Each event is a fact, it describes a state change that occurred to the entity (past tense!). As we all know, facts are indisputable and immutable. For example, suppose we had an application that saved a customer’s details. If we took an event sourcing approach, we would store every change made to that customer’s information as a stream, with the current state derived from a composition of the changes, much like a version control system does. Each individual change record in that stream would be an immutable, indisputable fact.

Having a stream of such events, it’s possible to find out what’s the current state of an entity by folding all events relating to that entity; note, however, that it’s not possible the other way round — when storing the current state only, we discard a lot of valuable historical information.

Event sourcing can peacefully co-exist with more traditional ways of storing state. A system typically handles a number of entity types (e.g. users, orders, products, …), and it’s quite possible that event sourcing is beneficial for only some of them. It’s important to remember that it’s not an all-or-nothing choice, but an additional possibility when it comes to choosing how state is managed in our application.

Storing events in Kafka

The first problem to solve is how we store events in Kafka. There are three possible strategies:

  1. Store all events for all entity types in a single topic (which is, of course, partitioned)
  2. Topic-per-entity-type, where we create a separate topic for all events related to a particular entity (e.g., a topic for all user-related events, one for product-related events, etc).
  3. Topic-per-entity, where we create separate topics for each user, each product, and so on.

Apart from low-cardinality entities, the third strategy (topic-per-entity) is not feasible. If each new user in the system would require the creation of a topic, we would end up with an unbounded number of topics. Any aggregations would also be very hard, such as indexing all users in a search engine, as it would require consuming a large amount of topics, which in addition wouldn’t all be known upfront.

Hence, we can choose between options one and two. Both have their pros and cons: with a single topic, it’s easier to get a global view of all events. On the other hand, with topic-per-entity-type, it’s possible to partition and scale each entity type stream separately. The choice between the two depends on the use case.

It is also possible to have both, at the cost of additional storage: derive entity type topics from the all-events topic.

In the remainder of the article we’ll assume we’re working with a single entity type and a single topic; however, it’s easy to generalise to multiple topics or entity types. For more on this, check out Martin Kleppmann’s recent blog post on the topic.

Basic event-sourcing storage operations

The most basic operation that we would expect from a data store that supports event sourcing is reading the “current” (folded) state of a particular entity. Typically, each entity has some kind of an ID. Given that ID, our storage system should return its current state.

The event log is the primary source of truth: the current state can always be derived from the stream of events for a particular entity. In order to do that, the storage engine needs a pure (side-effect free) function, taking the event and current state and returning the modified state: Event => State => State. Given such a function and an initial state value, the current state is a fold over the stream of events. (The state-modification function needs to be pure so that it can be freely applied multiple times to the same events.)

A naive implementation of the “read current state” operation in Kafka would stream all of the events from the topic, filter them to include only the events for the given id and fold them using the given function. If there’s a large number of events (and over time, the number of events only grows), this can be a slow and resource-consuming operation. Even if the result would be cached in-memory in a service node, it would still need to be periodically re-created, for example due to node failures or cache eviction.

Hence, we need a better way. That’s where Kafka Streams and state stores come into play. Kafka Streams applications run across a cluster of nodes, which jointly consume some topics. Each node is assigned a number of partitions of the consumed topics, just as with a regular Kafka consumer. However, Kafka Streams provides higher-level operations on the data, allowing much easier creation of derivative streams.

One such operation in Kafka Streams is folding a stream into a local store. Each local store contains data only from the partitions that are consumed by a given node. There are two local store implementations available out of the box: a persistent storage engine (RocksDB, used by default) and an in-memory store. Both implementations are fault-tolerant, so even if you decide to configure your applications to use the in-memory store, you will not lose data.

Coming back to event sourcing, we can fold the stream of events into the state store, keeping locally the “current state” of each entity from the partitions assigned to the node. If we are using the RocksDB implementation of the state store, we are only limited by disk space as to how many entities can be tracked on a single node—and of course we can always mitigate that scale problem by further partitioning the source topic.

Here’s how folding events into a local store looks like using the Java API (serde stands for serializer/deserializer):

StreamsBuilder builder = new StreamsBuilder();
builder.stream("my_entity_events", Consumed.with(keySerde, valueSerde))
  .groupByKey()
  // the folding function: should return the new state
  .reduce((currentState, event) -> ..., Materialized.<String, Long, KeyValueStore<Bytes, byte[]>as("my_entity_store").with(keySerde, valueSerde));
  .toStream(); // yields a stream of intermediate states
return builder;

For a full example, check out the orders microservices example in GitHub. In an event-sourcing system, you may want to retain data forever, rather than the default retention period of seven days. To get this done, set log.retention.bytes = -1 on the applicable topics.

Looking up the current state

We have created a state store containing the current states of all entities coming from partitions assigned to the node, but how to query it? If the query is local (same node), then it’s quite straightforward:

streams.store("my_entity_store", QueryableStoreTypes.keyValueStore()).get(entityId);

But what if we want to query for data which is present on another node? And how do we find out which node it is? Here, another feature recently introduced to Kafka comes in: interactive queries. Using them, it’s possible to query Kafka’s metadata and find out which node processes the topic partition for a given ID (this uses the topic partitioner behind the scenes):

metadataService.streamsMetadataForStoreAndKey("my_entity_store", entityId, keySerde);

Then it’s a matter of forwarding the request to the appropriate node. Note that how the inter-node communication is handled and implemented — is it REST, akka-remoting or any other way — is outside the scope of Kafka Streams. Kafka just allows accessing the state store, and gives information on which host a state store for a given ID is present.

Fail-over

State stores look nice, but what happens if a node fails? Re-creating the local state store for a partition might also be an expensive operation. It can cause increased latencies or failed requests for a long period of time due to Kafka Streams re-balancing (after a node is added or removed).

That’s why by default persistent state stores are logged: that is, all changes to the store are additionally written to a changelog-topic. This topic is compacted (we only need the latest entry for each ID, without the history of changes, as the history is kept in the events) and hence is as small as possible. Thanks to that, re-creating the store on another node can be much faster.

But that still might cause latencies on re-balancing. To reduce them even further, Kafka Streams has an option to keep a number of standby replicas (num.standby.replicas) for each storage. These replicas apply all the updates from the changelog topics as they come in, so if the primary fails Kafka Streams will fail over to the standby replica instantaneously..

Consistency

Using the default settings, Kafka provides at-least-once delivery. That is, in case of node failures, some messages might be delivered multiple times. It is possible, for example, that an event is applied to a state store twice, if the system failed after the state store changelog was written, but before the offset for that particular event was committed. That might not be a problem: our state-updating function (Event => State => State) might cope well with such situations. But it doesn’t have to; in that case, we can leverage Kafka’s exactly-once guarantees. These exactly-once guarantees only apply when reading and writing to Kafka topics, but that’s all that we are doing here: the action of updating the state store’s changelog and committing offsets are both writes to Kafka topics behind the scenes, and these can be done transactionally.

Hence, if our state-update function requires that, we can turn on exactly-once stream processing using a single configuration option: processing.guarantee. This introduces a performance penalty, but because the cost of a single commit is aggregated over a batch of messages the cost is comparatively small.

Listening for events

Now that we have the basics covered — querying and updating the “current state” of each entity — what about running side-effects? At some point, this will be necessary, for example to:

  • send notification e-mails
  • index entities in a search engine
  • call external services via REST (or SOAP, CORBA, etc. 😉 )

All of these tasks are in some way blocking and involve I/O (as is the nature of side-effects), so it’s probably not a good idea to execute them as part of the state-updating logic: that could cause an increased rate of failures in the “main” event loop and create a performance bottleneck.

Moreover, the state-updating logic function (Event => State => State) can be run multiple times (in case of failures or restarts), and most often we want to minimise the number of cases where side-effects for a given event are run multiple times.

Luckily, as we are working with Kafka topics, we have quite a lot of flexibility. It is possible to consume messages either before or after the stage where we update the state store, and these messages—before or after—can be consumed in an arbitrary way. Finally, we also have control if we want to run the side-effects at-least-once or at-most-once. At-least-once can be achieved by committing the offset of the consumed event-topic only after the side-effects complete successfully. Conversely, at-most-once, by committing the offsets before running the side-effects.

As to how the side-effects are run, there’s a number of options, depending on the use-case. First of all, we can define a Kafka Streams stage, which runs the side-effects for each event as part of the stream processing function. That’s quite easy to setup, however it isn’t a very flexible solution when it comes to retries, offset management and executing side-effects for many events concurrently. In more advanced cases like these, it might be more suitable simply to define the processing using a Kafka topic consumer.

There’s also a possibility that one event triggers other events — for example an “order” event might trigger “prepare for shipment” and “notify customer” events. This can also be implemented using a Kafka Streams stage.

Finally, if we’d like to store the events, or some data extracted from the events, in a database or search engine, such as ElasticSearch or PostgreSQL, we might use a Kafka Connect connector which will handle all of the topic-consuming details for us.

Creating views and projections

Usually the requirements of a system go beyond querying and handling only individual entity streams. Aggregations, combining a number of event streams, also need to be supported. Such aggregated streams are often called projections and when folded, can be used to create data views. Is it possible to implement this using Kafka?

Again, yes! Remember that at the basic level we are just dealing with a Kafka topic storing our events: and hence, we have all the power of “raw” Kafka consumers/producers, Kafka Streams combinator and even KSQL to define the projections. For example, with Kafka Streams we can write Java or Scala code to filter the stream, or do operations like map, groupByKey, or aggregate in time or session windows. Alternatively, we can opt to use KSQL, where you use a SQL-like language to express the same processing logic instead of having to write code.

Such streams can be persistently stored and made available for querying using state stores and interactive queries, just like we did with individual entity streams.

Going further

As the system evolves, to prevent the event stream from growing indefinitely, some form of compaction or storing “current state” snapshots might come in handy. That way, we could store only a number of the recent snapshots and the events that occured after them. It is straightforward to build this kind of functionality in Kafka using the mechanisms we’ve already covered, like consumers, state stores, and so on.

Summing up

While Kafka wasn’t originally designed with event sourcing in mind, its design as a data streaming engine with replicated topics, partitioning, state stores and streaming APIs is very flexible and lends itself perfectly to the task. Moreover, since there’s always a Kafka topic behind the scenes, we get the additional flexibility of being able to work either with high-level streaming APIs or low-level consumers for maximum development convenience and velocity.

About Apache Kafka’s Streams API

If you have enjoyed this article, you might want to continue with the following resources to learn more about Apache Kafka’s Streams API:

Join us for a 3-part online talk series for the ins and outs behind how KSQL works, and learn how to use it effectively to perform monitoring, security and anomaly detection, online data integration, application development, streaming ETL, and more. Past talks were recorded and can be watched on-demand. The third part in the series, Deploying and Operating KSQL, is on March 15th.

The post Event Sourcing Using Apache Kafka appeared first on Confluent.

No More Silos: How to Integrate your Databases with Apache Kafka and CDC

$
0
0

One of the most frequent questions and topics that I see come up on community resources such as StackOverflow, the Confluent Platform mailing list, and the Confluent Community Slack group, is getting data from a database into Apache Kafka®, and vice versa. Often it’s Oracle, SQL Server, DB2, etc—but regardless of the actual technology, the options for doing it are broadly the same. In this post we’ll look at each of those options and discuss the considerations around each. It may be obvious to readers, but it’s worth restating anyway: since this is Kafka—a streaming platform—we are talking about streaming integration of data, not just bulk static copies of the data.

Databases? How Twentieth Century…

Before we dive too deeply into this, bear in mind that you don’t always need a database. If you’re wanting to ingest data from a database into Kafka, how did that data get into the database in the first place? Would it be better to instead ingest the data from the original source directly into Kafka itself? If you’re streaming data from Kafka into a database, why are you landing it to a database? Is it to integrate with another system that’s consuming the data? Would that other system be better ingesting the data from Kafka instead? It’s important to challenge assumptions about how systems are built, as Kafka introduces what we refer to in the trade as a paradigm shift that we need to leverage. Why introduce a database into an architecture if we could use a streaming platform such as Kafka instead? Kafka persists data, Kafka can process and transform data with the Kafka Streams API and KSQL too. Sometimes you need a database; but not always. For more insight on this, check out these articles:

So anyway…assuming we’ve decided that we do indeed need a database, let’s look first at how we can get data from a database into Apache Kafka.

The two options to consider are using the JDBC connector for Kafka Connect, or using a log-based Change Data Capture (CDC) tool which integrates with Kafka Connect. If you’re considering doing something different, make sure you understand the reason for doing it, as the above are the two standard patterns generally followed – and for good reasons.

Kafka Connect

The Kafka Connect API is a core component of Apache Kafka, introduced in version 0.9. It provides scalable and resilient integration between Kafka and other systems.

Kafka Connect diagram

It is driven purely by configuration files, providing an easy integration point for developers.

{
  "name": "jdbc_source_mysql_foobar_01",
  "config": {
                "connector.class": "io.confluent.connect.jdbc.JdbcSourceConnector",
                "connection.url": "jdbc:mysql://localhost:3306/demo?user=rmoff&password=rmoff",
                "table.whitelist": "foobar",
                "mode": "incrementing",
                "incrementing.column.name": "foobar_pk"

        }
}

Kafka Connect can be run as a clustered process across multiple nodes, and handles all the tricky business of integration, including:

  • Scaleout of ingest and egress across nodes for greater throughput
  • Automatic restart and failover of tasks in the event of node failure
  • Automatic offset management
  • Automatic preservation of source data schema
  • Utilisation of data’s schema to create target objects (e.g. Hive tables when streaming to HDFS, RDBMS tables when streaming to a database)
  • Schema evolution and compatibility support (in conjunction with the Confluent Schema Registry)
  • Automatic serialisation and deserialisation of data
  • Single Message Transformations
  • Exactly once processing semantics (on supported connectors)

To use Kafka Connect you simply need a plugin that integrates with the technology that you’re interested in. Confluent Open Source ships with several of these plugins, including JDBC, HDFS, Elasticsearch, and S3. You can find dozens more plugins for Kafka Connect here.

JDBC plugin for Kafka Connect

The Confluent JDBC Connector for Kafka Connect enables you to stream data to and from Kafka and any RDBMS that supports JDBC (which is to say pretty much any). It can stream entire schemas or just individual tables. It can pull the entire contents (bulk), or do an incremental fetch of data that’s changed since the last poll using a numeric key column, an update timestamp, or both.

It’s a super-easy way to get started with streaming data into Kafka from databases. You can see an example of it in action in this article, streaming data from MySQL into Kafka. Also check out the quickstart and the full documentation. The JDBC Connector also gives you a way to stream data from Kafka into a database—see details and examples in the quickstart here.

Because the JDBC Connector uses the Kafka Connect API, it has several great features when it comes to streaming data from databases into Kafka:

  • Configuration-only interface for developers—no coding!
  • Schemas
    • The source database schema is preserved for use downstream in consuming Kafka applications
    • When streaming data from Kafka to a database, the connector will use the schema to execute the necessary DDL on the target to create the destination objects before streaming data to them
  • Kafka Connect can scale out the number of tasks to increase the throughput. For example, it can ingest data from multiple tables in parallel. Just make sure you don’t flatten your database with too many concurrent requests!
  • Database-specific dialects (in progress)

So the JDBC Connector is a great start, and is good for prototyping, for streaming smaller tables into Kafka, and streaming Kafka topics into a relational database. But (and there’s always a but!) here are some considerations to bear in mind:

  • The connector works by executing a query, over JDBC, against the source database. It does this to pull in all rows (bulk) or those that changed since previously (incremental). This query is executed at the interval defined in poll.interval.ms. Depending on the volumes of data involved, the physical database design (indexing, etc.), and other workload on the database, this may not prove to be the most scalable option.
  • If you want to use the connector for incremental streaming of data, you need to be able to identify changed rows. This means either an incrementing ID column, and/or a column with a timestamp indicating the last change. If these don’t exist then it can be difficult to get them added unless you own the schema for the source data. Sometimes data comes from “blackbox” applications in which you literally cannot change the tables. Other times the data belongs to another team who won’t entertain making a change to their application.
  • The JDBC Connector cannot fetch deleted rows. Because, how do you query for data that doesn’t exist? Depending on why you’re integrating the database with Kafka, deleted records may be required (for example, event-driven services which need to know when an entity no longer exists).
    • There are of course workarounds, some more elegant than others. One example is that espoused by Bjöprn Rost and Stewart Bryson (video / slides), using Oracle’s Flashback feature to actually detect deletes.
    • Other options I can imagine people proposing involve some kind of TRIGGER and a table, replicating in effect log-driven Change Data Capture, which we are going to discuss below.
    • My caution would be that any home brew solution has got to be both scalable and supported. A PoC is one thing, but a deployment to production quite another.

So the JDBC connector is definitely the best place to get started with integrating databases and Kafka. But, particularly for streaming data from a database into Kafka, we sometimes need a better option.

Log-based Change-Data-Capture (CDC) tools and Kafka

If you want to go “the whole hog” with integrating your database with Kafka, then log-based Change-Data-Capture (CDC) is the route to go. Done properly, CDC basically enables you to stream every single event from a database into Kafka. Broadly put, relational databases use a transaction log (also called a binlog or redo log depending on DB flavour), to which every event in the database is written. Update a row, insert a row, delete a row – it all goes to the database’s transaction log. CDC tools generally work by utilising this transaction log to extract at very low latency and low impact the events that are occurring on the database (or a schema/table within it).

Many CDC tools exist, serving a broad range of sources. Some specialise in broad coverage of source systems, others in just specific ones. The common factor uniting most of them is close integration with Apache Kafka and Confluent Platform. Being able to stream your data from a database not only into Kafka, but with support for things such as the preservation of schemas through the Schema Registry, is a defining factor of these CDC tools. Some are built using the Kafka Connect framework itself (and tend to offer a richer degree of integration), whilst others use the Kafka Producer API in conjunction with support for the Schema Registry etc.

CDC tools with support from the vendor and integration with Confluent Platform are (as of March 2018):

You can read more about CDC & Kafka in action at these articles:

So what’s the catch with CDC? There isn’t one, per se. CDC is low impact, low latency, and gives you full data fidelity. There are a few reasons why you may not use a CDC tool when integrating a database with Kafka, at least to start with:

  • CDC tools are more complex than the straightforward JDBC connector.
    • This complexity may well be worth it given your requirements, but be aware that you are adding more moving parts to your overall solution.
    • For rapid prototyping CDC can be overkill (and the JDBC connector fits well here)
    • More complex to set up at first, because of the nature of the integration with the relatively low level log files
    • Often requires administration access to the database for initial setup—can be a speedbump to rapid prototyping
  • Cost considerations: many CDC tools are commercial offerings (typically those that work with proprietary sources).

Conclusion

My general steer on CDC vs JDBC is that JDBC is great for prototyping, and fine low-volume workloads. Things to consider if using the JDBC connector:

  • Doesn’t give true CDC (capture delete records, want before/after record versions)
  • Latency in detecting new events
  • Impact of polling the source database continually (and balancing this with the desired latency)
  • Unless you’re doing a bulk pull from a table, you need to have an ID and/or timestamp that you can use to spot new records. If you don’t own the schema, this becomes a problem.

Also bear in mind what your requirements are for the data. Are you wanting to simply stream data from the database so that you can use it in a traditional analytics/ETL sense? Or are you building event-driven applications? The former gives you more leeway on how you implement. The latter almost certainly necessitates a log-based CDC approach, because you need not only every event (rather than just the state at an arbitrary point in time), and you also need delete events.

For analytics and ETL, streaming data in from a database is a powerful way to integrate existing systems with Kafka. The great thing is that because Apache Kafka decouples sources and targets, you can easily switch one out for another. For example, an iterative development could look like :

  1. Stream data from database into Kafka using JDBC Connector – prove the value of the data being in Kafka in realtime to drive applications and requirements. Typically done as a prototype/sandbox/skunk-works project.
  2. Once proven out, more formally adopt the source of data into Kafka, and consider the use of CDC. Existing consuming applications and targets stay untouched, because the data is still the same (just being streamed into Kafka using a different method)

Interested in Learning More?

If you’d like to learn more about the Confluent Platform, here are some resources for you:

Join us for a 3-part online talk series for the ins and outs behind how KSQL works, and learn how to use it effectively to perform monitoring, security and anomaly detection, online data integration, application development, streaming ETL, and more. 

The post No More Silos: How to Integrate your Databases with Apache Kafka and CDC appeared first on Confluent.

KSQL in Action: Enriching CSV Events with Data from RDBMS into AWS

$
0
0

Life would be simple if data lived in one place: one single solitary database to rule them all. Anything that needed to be joined to anything could be with a simple SQL JOIN command. But…back in the real world, we have myriad systems scattered across a landscape of log files, NoSQL, RDBMS, CSV, VSAM, IMS and more. What fun, what joy! Long live data silos? Well, no. Not anymore. With Apache Kafka® and KSQL we can integrate data from anywhere, in realtime, and transform it with filters, joins, and aggregations in-flight.

In this article we’ll see how we can take an event and enrich it in realtime. We’ll look up attributes relating to the events from data originating in a database. If you’re from a database or analytics background, we’re basically talking good ole’ denormalisation—resolving foreign key relationships.

Our source data is CSV files with order data, and relational data about the customers who have placed these orders. We’ll join these together, and then stream the results to a datastore from where we can run some analyses against them.

Set up Customer Data in MySQL

For our customer data, we’re going to use the excellent Mockaroo to generate some data records with a simple schema.

Whilst Mockaroo can also generate the necessary CREATE TABLE DDL, I’m including it separately here. This is because you need to amend it to declare a Primary Key (PK). If you don’t, the compacted topic will cause Kafka Connect to throw an error Got error produce response with correlation id 2538 on topic-partition asgard.demo.customers-0, retrying (2147483156 attempts left). Error: CORRUPT_MESSAGE.

Run this DDL to create the table in MySQL:

CREATE DATABASE demo;
USE demo;
DROP TABLE customers;
CREATE TABLE customers (
        id INT PRIMARY KEY,
        first_name VARCHAR(50),
        last_name VARCHAR(50),
        email VARCHAR(50),
        gender VARCHAR(50),
        comments VARCHAR(90)
);

Now populate it with the Mockeroo output:

curl -s "https://api.mockaroo.com/api/94201080?count=1000&key=ff7856d0" | mysql demo -uroot

(this assumes that you’ve signed up for a Mockaroo account to get access to the API; if not just download the .sql file from Mockaroo’s web UI and pipe it into MySQL instead)

Check the data has been created in MySQL:

mysql> select * from demo.customers limit 1;
+----+------------+-----------+----------------------------+--------+-----------------------------+
| id | first_name | last_name | email                      | gender | comments                    |
+----+------------+-----------+----------------------------+--------+-----------------------------+
|  1 | Kania      | Eggleson  | keggleson0@tripadvisor.com | Female | Multi-channelled web-enable |
+----+------------+-----------+----------------------------+--------+-----------------------------+
1 row in set (0.00 sec)

mysql> select count(*) from demo.customers;
+----------+
| count(*) |
+----------+
|     1000 |
+----------+
1 row in set (0.00 sec)

mysql>

Stream customer data from MySQL to Kafka

We’re going to use Debezium to stream the contents of the customers table, plus any changes as they occur, from MySQL into Kafka. Instead of a normal Kafka topic with standard retention policies, we’re going to use a compacted topic. We want to work with customer information not a stream of events per se, but instead as the value (name, email, etc) per key(in this case, customer id).

Create the topic, and set it as compacted:

$ kafka-topics --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic asgard.demo.customers
WARNING: Due to limitations in metric names, topics with a period ('.') or underscore ('_') could collide. To avoid issues it is best to use either, but not both.
Created topic "asgard.demo.customers".
$
$ kafka-configs --zookeeper localhost:2181 --entity-type topics --entity-name asgard.demo.customers --alter --add-config cleanup.policy=compact
Completed Updating config for entity: topic 'asgard.demo.customers'.

(The topic naming is set by Debezium, and is based on <server>.<database>.<table>)

Now set up the Kafka Connect Debezium connector. This will stream the current contents of the table into Kafka, and then use MySQL’s binlog to track any subsequent changes to the table’s data (including deletes) and stream those to Kafka too.

curl -i -X POST -H "Accept:application/json" \
    -H  "Content-Type:application/json" http://localhost:8083/connectors/ \
    -d '{
      "name": "mysql-demo-customers",
      "config": {
            "connector.class": "io.debezium.connector.mysql.MySqlConnector",
            "database.hostname": "localhost",
            "database.port": "3306",
            "database.user": "debezium",
            "database.password": "dbz",
            "database.server.id": "42",
            "database.server.name": "asgard",
            "table.whitelist": "demo.customers",
            "database.history.kafka.bootstrap.servers": "localhost:9092",
            "database.history.kafka.topic": "dbhistory.demo" ,
            "include.schema.changes": "true",
            "transforms": "unwrap,InsertTopic,InsertSourceDetails",
            "transforms.unwrap.type": "io.debezium.transforms.UnwrapFromEnvelope",
            "transforms.InsertTopic.type":"org.apache.kafka.connect.transforms.InsertField$Value",
            "transforms.InsertTopic.topic.field":"messagetopic",
            "transforms.InsertSourceDetails.type":"org.apache.kafka.connect.transforms.InsertField$Value",
            "transforms.InsertSourceDetails.static.field":"messagesource",
            "transforms.InsertSourceDetails.static.value":"Debezium CDC from MySQL on asgard"
       }
    }'

Check the status of the connector:

$ confluent status connectors|grep -v Writing| jq '.[]'|  xargs -I{connector} confluent status {connector}|  grep -v Writing| jq -c -M '[.name,.connector.state,.tasks[].state]|join(":|:")'|  column -s : -t|  sed 's/\"//g'|  sort
mysql-connector  |  RUNNING  |  RUNNING

If it’s not RUNNING then check the Connect stdout for errors.

Now let’s check that the topic has been populated:

$ kafka-avro-console-consumer \
--bootstrap-server localhost:9092 \
--property schema.registry.url=http://localhost:8081 \
--topic asgard.demo.customers \
--from-beginning --max-messages 1| jq '.'
{
  "id": 1,
  "first_name": {
    "string": "Kania"
  },
  "last_name": {
    "string": "Eggleson"
  },
  "email": {
    "string": "keggleson0@tripadvisor.com"
  },
  "gender": {
    "string": "Female"
  },
  "comments": {
    "string": "Multi-channelled web-enabled ability"
  },
  "messagetopic": {
    "string": "asgard.demo.customers"
  },
  "messagesource": {
    "string": "Debezium CDC from MySQL on asgard"
  }
}

We can also see the Change-Data-Capture (CDC) in action. Run the kafka-avro-console-consumer command from above, but without the --max-messages 1—you’ll get the contents of the topic, and then it will sit there, waiting for new messages. In a separate terminal, connect to MySQL and make a change to the data:

mysql> update demo.customers set first_name='Bob2' where id = 1;
Query OK, 1 row affected (0.02 sec)
Rows matched: 1  Changed: 1  Warnings: 0

mysql>

You should see almost instantaneously the changed record appear in the Kafka topic.

Streaming CSV data into Kafka

Data comes in all sorts of shapes and sizes, from a few bytes of Avro to hundreds of megabytes of XML files, and more. And for better or worse, CSV files are still used as a common interchange format of data, despite a lack of declared schema and difficult to parse field separators. But the great news about Apache Kafka is that with Kafka Connect you can integrate data from lots of sources—even CSV!

To stream CSV data into Kafka we can use the kafka-connect-spooldir plugin. This monitors a folder for a given pattern of file, and processes them, streaming the rows into Kafka and applying the supplied schema. It supports CSV, TSV, and JSON data. Note that it is only suitable for files that are complete; if the file is still being written to by another process then it wouldn’t be appropriate to use this.

To get the plugin simply clone the GitHub repository and build the code. This is done using using Maven—download it first if you don’t have it.

git clone https://github.com/jcustenborder/kafka-connect-spooldir.git
cd kafka-connect-spooldir
mvn clean package

Having built the plugin, add it to the Kafka Connect worker’s configuration. Locate the plugin.path configuration entry and add /full/path/to/kafka-connect-spooldir/target/kafka-connect-target/usr/share/kafka-connect/ to the list of paths.

Create three folders: one for the source data, one where the files will be placed once processed, and one for any files in error. Here I’m using /tmp—for real-world use you would amend this to your own path.

mkdir /tmp/source
mkdir /tmp/finished
mkdir /tmp/error

We’re using dummy data, again courtesy of Mockeroo. Download some sample CSV files, call them orders<x>.csv(where <x> is a varying suffix) and place them in the source folder. The folders and contents should look like this:

$ ls -l /tmp/source/ /tmp/finished/ /tmp/error/
/tmp/error/:

/tmp/finished/:

/tmp/source/:
total 5224
-rw-r--r--@ 1 Robin  staff  440644  9 Feb 22:55 orders.csv
-rw-r--r--@ 1 Robin  staff  440644  9 Feb 23:37 orders1.csv
-rw-r--r--@ 1 Robin  staff  440644  9 Feb 23:39 orders2.csv

The great thing about the SpoolDir connector is that is applies a schema to the data, ensuring life is happy for those downstream in Kafka wanting to make use of the data. You can enter the schema by hand (and you do know the schema, right?), or you can use a utility that ships with the plugin to automagically determine it. To do this, create a temporary config file (e.g. /tmp/spool_conf.tmp) as follows:

input.path=/tmp/source
finished.path=/tmp/finished
csv.first.row.as.header=true

Run the utility, specifying the sample input file, and also the field name(s) that is to be the key:

# Run this from the folder in which you built kafka-connect-spooldir
$ export CLASSPATH="$(find target/kafka-connect-target/usr/share/kafka-connect/kafka-connect-spooldir/ -type f -name '*.jar' | tr '\n' ':')"
$ kafka-run-class com.github.jcustenborder.kafka.connect.spooldir.SchemaGenerator -t csv -f /tmp/source/orders.csv -c /tmp/spool_conf.tmp -i order_id
#Configuration was dynamically generated. Please verify before submitting.
#Wed Mar 14 16:01:08 GMT 2018
csv.first.row.as.header=true
value.schema={"name"\:"com.github.jcustenborder.kafka.connect.model.Value","type"\:"STRUCT","isOptional"\:false,"fieldSchemas"\:{"order_id"\:{"type"\:"STRING","isOptional"\:true},"customer_id"\:{"type"\:"STRING","isOptional"\:true},"order_ts"\:{"type"\:"STRING","isOptional"\:true},"product"\:{"type"\:"STRING","isOptional"\:true},"order_total_usd"\:{"type"\:"STRING","isOptional"\:true}}}
input.path=/tmp/source
key.schema={"name"\:"com.github.jcustenborder.kafka.connect.model.Key","type"\:"STRUCT","isOptional"\:false,"fieldSchemas"\:{"order_id"\:{"type"\:"STRING","isOptional"\:true}}}
finished.path=/tmp/finished

Now define the connector. The configuration items are fairly obvious, requiring only a couple of notes:

  • The input.file.pattern is a regex based on the filenames within the specified `input.path` (not the entire pathname as you may assume).
  • The key.schema and value.schema come from the output of the utility above.

    Make sure to escape the quotation marks in the schema correctly in the JSON you send with curl (replace \: with :and replace " with \") (which you can do by piping the above kafka-run-class statement through sedsed 's/\\:/:/g'|sed 's/\"/\\\"/g').

    I’ve also manually changed the STRING to INT64 for the ID columns, and marked all the fields as isOptional=false:

$ curl -i -X POST -H "Accept:application/json" \
    -H  "Content-Type:application/json" http://localhost:8083/connectors/ \
    -d '{
  "name": "csv-source-orders",
  "config": {
    "tasks.max": "1",
    "connector.class": "com.github.jcustenborder.kafka.connect.spooldir.SpoolDirCsvSourceConnector",
    "input.file.pattern": "^orders.*.csv$",
    "input.path": "/tmp/source",
    "finished.path": "/tmp/finished",
    "error.path": "/tmp/error",
    "halt.on.error": "false",
    "topic": "orders",
    "value.schema":"{\"name\":\"com.github.jcustenborder.kafka.connect.model.Value\",\"type\":\"STRUCT\",\"isOptional\":false,\"fieldSchemas\":{\"order_id\":{\"type\":\"INT64\",\"isOptional\":false},\"customer_id\":{\"type\":\"INT64\",\"isOptional\":false},\"order_ts\":{\"type\":\"STRING\",\"isOptional\":false},\"product\":{\"type\":\"STRING\",\"isOptional\":false},\"order_total_usd\":{\"type\":\"STRING\",\"isOptional\":false}}}",
    "key.schema":"{\"name\":\"com.github.jcustenborder.kafka.connect.model.Key\",\"type\":\"STRUCT\",\"isOptional\":false,\"fieldSchemas\":{\"order_id\":{\"type\":\"INT64\",\"isOptional\":false}}}",
    "csv.first.row.as.header": "true"
  }
}'

(N.B. this doesn’t handle the timestamp column parsing, instead bringing it in as a VARCHAR)

Check that the connector’s running:

$ curl -s "http://localhost:8083/connectors"| jq '.[]'| xargs -I{connector_name} curl -s "http://localhost:8083/connectors/"{connector_name}"/status"| jq -c -M '[.name,.connector.state,.tasks[].state]|join(":|:")'| column -s : -t| sed 's/\"//g'| sort
csv-source-orders     |  RUNNING  |  RUNNING
mysql-demo-customers  |  RUNNING  |  RUNNING

Assuming all of that has worked, you should see that all the source files are now in finished:

ls -l /tmp/source/ /tmp/finished/ /tmp/error/
/tmp/error/:

/tmp/finished/:
total 168
-rw-r--r--  1 Robin  wheel  27834 14 Mar 16:00 orders.csv
-rw-r--r--  1 Robin  wheel  27758 14 Mar 16:01 orders1.csv
-rw-r--r--  1 Robin  wheel  27894 14 Mar 16:01 orders2.csv

/tmp/source/:

And then the proof is in the pudding: do we have messages on our Kafka topic?

$ kafka-avro-console-consumer \
    --bootstrap-server localhost:9092 \
    --property schema.registry.url=http://localhost:8081 \
    --property print.key=true \
    --topic orders \
    --from-beginning \
    --max-messages 1|jq '.'
{
  "order_id": 1
}
{
  "order_id": 1,
  "customer_id": 827,
  "order_ts": "2018-02-28T04:40:50Z",
  "product": "Macaroons - Homestyle Two Bit",
  "order_total_usd": "3.30"
}

Yes, we do! \o/

Using KSQL to Enrich CSV data in Kafka

From the KSQL command prompt, we’ll first declare the stream of events (orders), using the Kafka topic populated from the CSV files:

ksql> CREATE STREAM orders WITH (KAFKA_TOPIC='orders',VALUE_FORMAT='avro');

It’s easy to check that we’ve got data—note the use of SET 'auto.offset.reset' = 'earliest'; to tell KSQL to process all data in the topic, rather than the default which to only new data:

ksql> SET 'auto.offset.reset' = 'earliest';
Successfully changed local property 'auto.offset.reset' from 'null' to 'earliest'
ksql> SELECT * FROM orders LIMIT 1;
1521209173235 | - | 1 | 288 | 2018-03-05T07:30:55Z | Wine - Sherry Dry Sack, William | 1.53
LIMIT reached for the partition.
Query terminated

Now let’s look at the customers. We are going to declare this as a table in KSQL, because it is a set of values for a given key (as opposed to a stream of events, which is what the orders are). But, we need to be careful with the key. Before declaring the object (CREATE TABLE) we can use the PRINT command to inspect the topic:

ksql> PRINT 'asgard.demo.customers' FROM BEGINNING;
Format:AVRO
16/03/18 14:01:27 GMT, +�, {"id": 245, "first_name": "Sergent", "last_name": "Greenmon", "email": "sgreenmon6s@wordpress.com", "gender": "Male", "comments": "Synergized optimizing pricing structure", "messagetopic": "asgard.demo.customers", "messagesource": "Debezium CDC from MySQL on asgard"}

Note the special characters in the record key (the , +�, after the timestamp, before the message payload). This is because the key is actually an Avro key — and KSQL can’t handle Avro keys yet, so blanks it out. We can verify that this is the case by looking at the raw message and its key:

$ kafka-avro-console-consumer \
--bootstrap-server localhost:9092 \
--property schema.registry.url=http://localhost:8081 \
--topic asgard.demo.customers --property print.key=true --max-messages=1 --from-beginning
{"id":1}        {"id":1,"first_name":{"string":"Kania"},"last_name":{"string":"Eggleson"},"email":{"string":"keggleson0@tripadvisor.com"},"gender":{"string":"Female"},"comments":{"string":"Multi-channelled web-enabled ability"},"messagetopic":{"string":"asgard.demo.customers"},"messagesource":{"string":"Debezium CDC from MySQL on asgard"}}
Processed a total of 1 messages

We could workaround this upstream in the Kafka Connect config (by using "key.converter":"org.apache.kafka.connect.storage.StringConverter" in the configuration), or we could just use KSQL!

ksql> SET 'auto.offset.reset' = 'earliest';
ksql> CREATE STREAM CUSTOMERS_SRC WITH (KAFKA_TOPIC='asgard.demo.customers',VALUE_FORMAT='AVRO');
ksql> CREATE STREAM CUSTOMERS_REKEYED AS SELECT * FROM CUSTOMERS_SRC PARTITION BY ID;

The above statement takes the source topic which is flowing through from MySQL via Debezium, and explicitly partitions it on the supplied key—the ID column. KSQL does this and the resulting topic is keyed as we want, and using a simple String for the key this time:

ksql> PRINT 'CUSTOMERS_REKEYED' FROM BEGINNING;
Format:AVRO
02/03/18 23:48:05 GMT, 5, {"ID": 5, "FIRST_NAME": "Marion", "LAST_NAME": "Micklem", "EMAIL": "mmicklem4@mail.ru", "GENDER": "Male", "COMMENTS": "Reactive clear-thinking functionalities", "MESSAGETOPIC": "asgard.demo.customers", "MESSAGESOURCE": "Debezium CDC from MySQL on asgard"}

Inspect the message—we can’t use kafka-avro-console-consumer because it assumes that the key is also Avro-encoded, which it isn’t this time. Instead, we’ll use the excellent kafkacat tool:

$ kafkacat -C -K: -b localhost:9092 -f 'Key: %k\nValue: %s\n\n' -t CUSTOMERS_REKEYED -c1
Key: 5
Value:
MarionMicklem"mmicklem4@mail.rMaleNReactive clear-thinking functionalities*asgard.demo.customersBDebezium CDC from MySQL on asgard

We can now use the correctly-keyed topic for our KSQL table:

ksql> CREATE TABLE CUSTOMERS WITH (KAFKA_TOPIC='CUSTOMERS_REKEYED', VALUE_FORMAT='AVRO', KEY='ID');

Check that the table’s declared key (ID) matches that of the Kafka message key:

ksql> SELECT ROWKEY,ID FROM CUSTOMERS LIMIT 5;
5 | 5
6 | 6
10 | 10
12 | 12
15 | 15
LIMIT reached for the partition.
Query terminated

Now the bit we’ve all been waiting for…enrich the stream of inbound orders data with customer data from MySQL!

ksql> SELECT O.ORDER_TS, O.PRODUCT, O.ORDER_TOTAL_USD, \
      C.ID, C.FIRST_NAME, C.LAST_NAME, C.EMAIL \
      FROM ORDERS O \
      LEFT OUTER JOIN CUSTOMERS C \
      ON O.CUSTOMER_ID = C.ID \
      LIMIT 5;
2018-03-13T01:50:53Z | Onions - Spanish | 9.44 | 115 | Alexandr | Willcot | awillcot36@facebook.com
2018-03-04T07:58:10Z | Halibut - Whole, Fresh | 5.11 | 929 | Ulick | Dumberell | udumberellps@ucla.edu
2018-02-09T19:11:15Z | Beef Wellington | 7.33 | 632 | Jennie | McMichell | jmcmichellhj@miitbeian.gov.cn
2018-03-11T15:39:49Z | Chocolate Eclairs | 1.45 | 270 | Margareta | Kerfod | mkerfod7h@nhs.uk
2018-03-04T23:27:04Z | Wine - George Duboeuf Rose | 6.68 | 117 | Duky | Raden | draden38@marketwatch.com
LIMIT reached for the partition.
Query terminated

(I’m using the \ line-continuation character to make it easier to read the KSQL statements, but you can put it all on one line if you want)

We can persist this streaming query with a CREATE STREAM statement:

ksql> CREATE STREAM ORDERS_ENRICHED AS \
     SELECT O.ORDER_TS, O.PRODUCT, O.ORDER_TOTAL_USD, \
      C.ID, C.FIRST_NAME, C.LAST_NAME, C.EMAIL \
      FROM ORDERS O \
      LEFT OUTER JOIN CUSTOMERS C \
      ON O.CUSTOMER_ID = C.ID \
 Message
----------------------------
 Stream created and running
----------------------------

This is a continuous query that executes in the background until explicitly terminated by the user. In effect, these are stream processing applications, and all we need to create them is SQL! Here all we’ve done is an enrichment (joining two sets of data), but we could easily add predicates to the data (simply include a WHERE clause), or even aggregations.

You can see which queries are running with the SHOW QUERIES; statement. All queries will pause if the KSQL server stops, and restart automagically when the KSQL server starts again.

The DESCRIBE EXTENDED command can be used to see information about the derived stream such as the one created above. As well as simply the columns involved, we can see information about the underlying topic, and run-time stats such as the number of messages processed and the timestamp of the most recent one.

ksql> DESCRIBE EXTENDED ORDERS_ENRICHED;

Type                 : STREAM
Key field            : O.CUSTOMER_ID
Timestamp field      : Not set - using <ROWTIME>
Key format           : STRING
Value format         : AVRO
Kafka output topic   : ORDERS_ENRICHED (partitions: 4, replication: 1)

 Field           | Type
---------------------------------------------
 ROWTIME         | BIGINT           (system)
 ROWKEY          | VARCHAR(STRING)  (system)
 ORDER_TS        | VARCHAR(STRING)
 PRODUCT         | VARCHAR(STRING)
 ORDER_TOTAL_USD | VARCHAR(STRING)
 ID              | INTEGER
 FIRST_NAME      | VARCHAR(STRING)
 LAST_NAME       | VARCHAR(STRING)
 EMAIL           | VARCHAR(STRING)
---------------------------------------------

Queries that write into this STREAM
-----------------------------------
id:CSAS_ORDERS_ENRICHED - CREATE STREAM ORDERS_ENRICHED AS SELECT O.ORDER_TS,O.PRODUCT,O.ORDER_TOTAL_USD,C.ID, C.FIRST_NAME, C.LAST_NAME, C.EMAIL FROM ORDERS O LEFT OUTER JOIN CUSTOMERS C ON O.CUSTOMER_ID = C.ID LIMIT 5;

For query topology and execution plan please run: EXPLAIN <QueryId>

Local runtime statistics
------------------------
messages-per-sec:     15.08   total-messages:      1500     last-message: 14/03/18 16:15:07 GMT
 failed-messages:         0 failed-messages-per-sec:         0      last-failed:       n/a
(Statistics of the local KSQL server interaction with the Kafka topic ORDERS_ENRICHED)

Underneath every persisted KSQL stream or table query (i.e. CSAS or CTAS) is a Kafka topic. This is just a Kafka topic as any other:

$ kafka-avro-console-consumer \
   --bootstrap-server localhost:9092 \
   --property schema.registry.url=http://localhost:8081 \
   --topic ORDERS_ENRICHED --max-messages=1 --from-beginning|jq '.'
{
 "ORDER_TS": {
   "string": "2018-03-13T01:50:53Z"
 },
 "PRODUCT": {
   "string": "Onions - Spanish"
 },
 "ORDER_TOTAL_USD": {
   "string": "9.44"
 },
 "ID": {
   "int": 115
 },
 "FIRST_NAME": {
   "string": "Alexandr"
 },
 "LAST_NAME": {
   "string": "Willcot"
 },
 "EMAIL": {
   "string": "awillcot36@facebook.com"
 }
}
Processed a total of 1 messages

Streaming the Enriched Data to S3 for Visual Analysis

We’ve seen how easy it is to ingest data from multiple sources—whether flat-file or RDBMS—and join it effortlessly. Now let’s see how we can stream it to a target datastore in order to built analytics on it.

S3 is Amazon’s ubiquitous object store, used extremely widely for both long-term storage of data for analytics, as well as operational data files. Confluent Platform ships with a Kafka Connect connector for S3, meaning that any data that is in Kafka can be easily streamed to S3. The connector supports exactly-once delivery semantics, as well as useful features such as customisable partitioning.

To set up the S3 connector you just need your bucket name, region, and your AWS access keys that have permission to write to the bucket. You can make the credentials available to the connector in several ways, the simplest being to set the required environment variables before launching the Connect worker.

export AWS_ACCESS_KEY_ID=XXXXXXXXXXXXXXXXXXXX
export AWS_SECRET_ACCESS_KEY=YYYYYYYYY/YYYYYYYYY

Restart the Connect worker to pick up the new environment variables:

confluent stop connect
confluent start connect

Now create the connector:

curl -i -X POST -H "Accept:application/json" \
    -H  "Content-Type:application/json" http://localhost:8083/connectors/ \
    -d '{
  "name": "s3-sink-orders",
  "config": {
    "connector.class": "io.confluent.connect.s3.S3SinkConnector",
    "key.converter":"org.apache.kafka.connect.storage.StringConverter",
    "tasks.max": "1",
    "topics": "ORDERS_ENRICHED",
    "s3.region": "us-east-2",
    "s3.bucket.name": "rmoff-demo-orders-02",
    "s3.part.size": "5242880",
    "flush.size": "3",
    "storage.class": "io.confluent.connect.s3.storage.S3Storage",
    "format.class": "io.confluent.connect.s3.format.avro.AvroFormat",
    "schema.generator.class": "io.confluent.connect.storage.hive.schema.DefaultSchemaGenerator",
    "partitioner.class": "io.confluent.connect.storage.partitioner.DefaultPartitioner",
    "schema.compatibility": "NONE"
    }
}'

One thing to note is that we’re using "key.converter":"org.apache.kafka.connect.storage.StringConverter", because the messages that KSQL is writing are not keyed with Avro, but String. Without this override Kafka Connect will use the default worker settings (which in my case are Avro), and the task will fail with a org.apache.kafka.connect.errors.DataException, and error detail Error deserializing Avro message for id -1 Unknown magic byte!.

Check that the connector’s running:

$ curl -s "http://localhost:8083/connectors"| jq '.[]'| xargs -I{connector_name} curl -s "http://localhost:8083/connectors/"{connector_name}"/status"| jq -c -M '[.name,.connector.state,.tasks[].state]|join(":|:")'| column -s : -t| sed 's/\"//g'| sort
csv-source-orders     |  RUNNING  |  RUNNING
mysql-demo-customers  |  RUNNING  |  RUNNING
s3-sink-orders        |  RUNNING  |  RUNNING

Go to S3 and you’ll see the files now exist! You can use the web GUI, or the aws cli:

$ aws s3 ls rmoff-demo-orders-02/topics/ORDERS_ENRICHED/partition=0/
2018-03-16 15:14:31        878 ORDERS_ENRICHED+0+0000000000.avro
2018-03-16 15:14:32        891 ORDERS_ENRICHED+0+0000000003.avro
2018-03-16 15:14:32        882 ORDERS_ENRICHED+0+0000000006.avro
2018-03-16 15:14:33        897 ORDERS_ENRICHED+0+0000000009.avro
2018-03-16 15:14:34        893 ORDERS_ENRICHED+0+0000000012.avro

People use S3 for a variety of reasons, and being able to stream data into it from Kafka via the Kafka Connect S3 connector is really useful. In this example here we can take the data, and use AWS’s Quicksight to do some analytical visualisation on top of it, first exposing the data via Athena and auto-discovered using Glue.

Amazon Quicksight

Summary

Apache Kafka and KSQL make for a powerful toolset for integrating and enriching data from one or more sources. All that’s needed is configuration files and SQL—not a single line of code was written in creating this article! Kafka Connect makes it easy to ingest data from numerous sources, as well as stream data from Kafka topics to many different targets.

Itching to get started with KSQL and see what it can do?

To read more about building streaming data pipelines with Apache Kafka and KSQL, check out the following articles:

The post KSQL in Action: Enriching CSV Events with Data from RDBMS into AWS appeared first on Confluent.

We ❤️ syslogs: Real-time syslog Processing with Apache Kafka and KSQL—Part 1: Filtering

$
0
0

syslog is one of those ubiquitous standards on which much of modern computing runs. Built into operating systems such as Linux, it’s also commonplace in networking and IoT devices like IP cameras. It provides a way for streaming log messages, along with metadata such as the source host, severity of the message, and so on. Sometimes the target is simply a local logfile, but more often it’s a centralised syslog server which in turn may log or process the messages further.

As a high-performance, distributed streaming platform, Apache Kafka® is a great tool for centralised ingestion of syslog data. Since Apache Kafka also persists data and supports native stream processing we don’t need to land it elsewhere before we can utilise the data. You can stream syslog data into Kafka in a variety of ways, including through Kafka Connect for which there is a dedicated syslog plugin.

In this post, we’re going to see how KSQL can be used to process syslog messages as they arrive in realtime. KSQL is the SQL streaming engine for Apache Kafka. Using just SQL, executing interactively or as a deployed application, we can filter, enrich and aggregate streams of data in Kafka. Processed data from KSQL goes back onto Kafka, meaning that we can easily filter and cleanse data arriving in Kafka for use up-stream in multiple applications requiring the same data.

Diagram showing KSQL filtering inbound syslog data

Filter syslog data with KSQL

Getting started with syslog and stream processing in KSQL is simple. First of all, download and install Confluent Platform, and then install KSQL following the directions in the GitHub repo.  (As of Confluent Platform 4.1, KSQL will be included in the platform distribution.) You’ll also need to install and configure the syslog plugin for Kafka Connect, and then configure your syslog sources to send their data to it. How you do this depends on your syslog client implementation—for rsyslog see here.

Assuming you’re using the syslog topic for your inbound data, we’ll can start by simply inspecting the data on the topic, from the KSQL prompt (run ksql-cli local to launch):

ksql> PRINT 'syslog' FROM BEGINNING;
Format:AVRO
12/03/18 12:21:06 GMT, */192.168.10.250:47823, {"date": null, "facility": 1, "host": "I", "level": 6, "message": "I ❤  logs\n", "charset": "UTF-8", "remote_address": "/192.168.10.250:47823", "hostname": "proxmox01.moffatt.me"}

If you leave this running you’ll see all new events streaming to the console. Press Ctrl-C to return to the KSQL prompt.

Format:AVRO

We can now declare a KSQL stream over the topic, using the format detected:

ksql> CREATE STREAM SYSLOG WITH (KAFKA_TOPIC='syslog',VALUE_FORMAT='AVRO');

 Message
----------------
 Stream created
----------------

Using standard SQL commands, we can query and manipulate the stream of events.

ksql> SELECT HOSTNAME,MESSAGE FROM SYSLOG;
proxmox01.moffatt.me | I ❤  logs
proxmox01.moffatt.me | I still ❤  logs

This is a continuous query, and will continue to show new data as it arrives. Press Ctrl-C to cancel and return to the KSQL prompt.

Apache Kafka persists data, so not only can KSQL show current and future inbound messages—we can also query the past! To do this we’ll instruct KSQL to process messages from the beginning of the topic:

ksql> SET 'auto.offset.reset' = 'earliest';
Successfully changed local property 'auto.offset.reset' from 'null' to 'earliest'

Now when you re-run the same SELECT statement as above, you see the full contents of the syslog topic:

ksql> SELECT HOSTNAME,MESSAGE FROM SYSLOG;
localhost | X
localhost | foo I love logs
localhost | I ❤  logs
proxmox01.moffatt.me | I ❤  logs
proxmox01.moffatt.me | I still ❤  logs

I’m going to cut over to some real syslog data now, so that we can explore the potential of what we can do with KSQL. My syslog data is streamed from my home network—a few servers and containers, some network APs, and a bunch of mobile devices.

ksql> SELECT TIMESTAMPTOSTRING(DATE, 'yyyy-MM-dd HH:mm:ss') AS SYSLOG_TS, HOST, MESSAGE FROM SYSLOG;
2018-03-12 13:30:59 | rpi-03 | rpi-03 sshd[30105]: Invalid user oracle from 185.55.218.153
2018-03-12 13:30:59 | rpi-03 | rpi-03 sshd[30105]: input_userauth_request: invalid user oracle [preauth]
2018-03-12 13:30:59 | rpi-03 | rpi-03 sshd[30105]: Received disconnect from 185.55.218.153: 11: Bye Bye [preauth]
2018-03-12 13:31:00 | rpi-03 | rpi-03 sshd[30109]: reverse mapping checking getaddrinfo for host3.artegix.info [185.55.218.153] failed - POSSIBLE BREAK-IN ATTEMPT!
2018-03-12 13:31:01 | rpi-03 | rpi-03 sshd[30117]: Invalid user test from 185.55.218.153
2018-03-12 13:31:01 | rpi-03 | rpi-03 sshd[30117]: input_userauth_request: invalid user test [preauth]
2018-03-12 13:31:01 | rpi-03 | rpi-03 sshd[30117]: Received disconnect from 185.55.218.153: 11: Bye Bye [preauth]
2018-03-12 13:31:02 | rpi-03 | rpi-03 sshd[30121]: reverse mapping checking getaddrinfo for host3.artegix.info [185.55.218.153] failed - POSSIBLE BREAK-IN ATTEMPT!
2018-03-12 13:31:02 | rpi-03 | rpi-03 sshd[30121]: Invalid user test from 185.55.218.153
2018-03-12 13:31:05 | ("BZ2,24a43c000000,v3.7.40.6115") | ("BZ2,24a43c000000,v3.7.40.6115") syslog: dpi.dpi_stainfo_notify(): dpi not enable
2018-03-12 13:31:05 | ("BZ2,24a43c000000,v3.7.40.6115") | ("BZ2,24a43c000000,v3.7.40.6115") hostapd: ath1: STA xx:xx:xx:xx:xx:xx IEEE 802.11: associated
2018-03-12 13:31:06 | ("U7PG2,f09fc2000000,v3.7.40.6115") | ("U7PG2,f09fc2000000,v3.7.40.6115") hostapd: ath3: STA xx:xx:xx:xx:xx:xx IEEE 802.11: disassociated
2018-03-12 13:31:06 | ("U7PG2,f09fc2000000,v3.7.40.6115") | ("U7PG2,f09fc2000000,v3.7.40.6115") libubnt[9577]: dpi.dpi_stainfo_notify(): dpi not enable
2018-03-12 13:31:06 | ("U7PG2,f09fc2000000,v3.7.40.6115") | ("U7PG2,f09fc2000000,v3.7.40.6115") libubnt[9577]: wevent.ubnt_custom_event(): EVENT_STA_LEAVE ath3: xx:xx:xx:xx:xx:xx / 0
2018-03-12 13:31:06 | ("BZ2,24a43c000000,v3.7.40.6115") | ("BZ2,24a43c000000,v3.7.40.6115") hostapd: ath1: STA xx:xx:xx:xx:xx:xx RADIUS: starting accounting session 5A9BFF48-00000286
2018-03-12 13:31:06 | ("BZ2,24a43c000000,v3.7.40.6115") | ("BZ2,24a43c000000,v3.7.40.6115") hostapd: ath1: STA xx:xx:xx:xx:xx:xx WPA: pairwise key handshake completed (RSN)
2018-03-12 13:31:06 | ("BZ2,24a43c000000,v3.7.40.6115") | ("BZ2,24a43c000000,v3.7.40.6115") syslog: wevent.ubnt_custom_event(): EVENT_STA_JOIN ath1: xx:xx:xx:xx:xx:xx / 4
2018-03-12 13:33:35 | proxmox01 | proxmox01 kernel: [30936072.607801] audit: type=1400 audit(1520861615.501:3062182): apparmor="DENIED" operation="ptrace" profile="docker-default" pid=26854 comm="node" requested_mask="trace" denied_mask="trace" peer="docker-default"
2018-03-12 13:33:38 | proxmox01 | proxmox01 kernel: [30936075.188047] audit: type=1400 audit(1520861618.081:3062183): apparmor="DENIED" operation="ptrace" profile="docker-default" pid=26854 comm="node" requested_mask="trace" denied_mask="trace" peer="docker-default"----

This is just a small sample of what’s in there, but we can spot a few things going on:

  • Login attacks against one of my public-facing servers
  • Wifi access point client connections/disconnections
  • Linux security module action logs

We’re going to see now how we can extract just certain logs of interest—in this case, the login attacks.

Note 

We’re deliberately not going to try and do this closer to the originator (e.g. by changing the syslog config on the source) because this ‘premature optimisation’ of our data reduces its overall fidelity and prevents us doing other work with it.

For example: we still want to retain all our syslog data for long-term storage—perhaps for audit purposes, perhaps for purposes we don’t even know yet—and if we cut down the scope of the data too early we can’t do this.

To filter out our syslog data (both that already on the topic, that arriving now, and all that arrives in the future), we just use a SQL predicate clause:

ksql> SELECT TIMESTAMPTOSTRING(DATE, 'yyyy-MM-dd HH:mm:ss') AS SYSLOG_TS, HOST, MESSAGE \
FROM SYSLOG \
WHERE HOST='rpi-03' AND MESSAGE LIKE '%Invalid user%'\
LIMIT 5;
2018-03-04 15:14:24 | rpi-03 | rpi-03 sshd[24150]: Invalid user mini from 114.130.4.16
2018-03-04 15:21:49 | rpi-03 | rpi-03 sshd[24200]: Invalid user admin from 103.99.0.209
2018-03-04 15:21:58 | rpi-03 | rpi-03 sshd[24204]: Invalid user support from 103.99.0.209
2018-03-04 15:22:06 | rpi-03 | rpi-03 sshd[24208]: Invalid user user from 103.99.0.209
2018-03-04 15:22:23 | rpi-03 | rpi-03 sshd[24216]: Invalid user 1234 from 103.99.0.209
LIMIT reached for the partition.
Query terminated

(Note I’m using the \ line continuation character, to make the statement more readable. You can enter it all on a single line if you’d like)

This is pretty useful for being able to quickly interrogate and inspect the logs. But let’s see something even more useful! We can persist this data, into a stream which holds not only the past logs filtered, but also every new log that is received. To do this, you just add the CREATE STREAM foo AS (often referred to as CSAS) to the front of your query:

ksql> CREATE STREAM SYSLOG_INVALID_USERS AS \
SELECT * \
FROM SYSLOG \
WHERE HOST='rpi-03' AND MESSAGE LIKE '%Invalid user%';

 Message
----------------------------
 Stream created and running
----------------------------

We’ve created a derived stream, which can be queried just as any other object in KSQL:

ksql> SELECT * FROM  SYSLOG_INVALID_USERS LIMIT 1;
1520176464386 | //192.168.10.105:38254 | 1520176464000 | 4 | rpi-03 | 6 | rpi-03 sshd[24150]: Invalid user mini from 114.130.4.16 | UTF-8 | /192.168.10.105:38254 | rpi-03.moffatt.me
LIMIT reached for the partition.
Query terminated

Under the covers, KSQL has actually created a Kafka topic and is populating this topic in real-time with any messages that match the criteria that we’ve defined against the source topic. We can see the new topic, and the messages in it—it’s just a Kafka topic:

$ kafka-topics --zookeeper localhost:2181 --list|grep SYSLOG
SYSLOG_INVALID_USERS
$
$ kafka-avro-console-consumer \
 --bootstrap-server proxmox01.moffatt.me:9092 \
 --property schema.registry.url=http://proxmox01.moffatt.me:8081 \
 --topic SYSLOG_INVALID_USERS --max-messages=1 --from-beginning|jq '.'
 {
   "DATE": {
     "long": 1520176464000
   },
   "FACILITY": {
     "int": 4
   },
   "HOST": {
     "string": "rpi-03"
   },
   "LEVEL": {
     "int": 6
   },
   "MESSAGE": {
     "string": "rpi-03 sshd[24150]: Invalid user mini from 114.130.4.16"
   },
   "CHARSET": {
     "string": "UTF-8"
   },
   "REMOTE_ADDRESS": {
     "string": "/192.168.10.105:38254"
   },
   "HOSTNAME": {
     "string": "rpi-03.moffatt.me"
   }
 }
Processed a total of 1 messages

To track the throughput of the new stream, as well as see more details such as the underlying topic name, columns, etc use the DESCRIBE EXTENDED command:

ksql> DESCRIBE EXTENDED SYSLOG_INVALID_USER_LOGIN;

Type                 : STREAM
Key field            :
Timestamp field      : Not set - using <ROWTIME>
Key format           : STRING
Value format         : AVRO
Kafka output topic   : SYSLOG_INVALID_USER_LOGIN (partitions: 4, replication: 1)

 Field          | Type
--------------------------------------------
 ROWTIME        | BIGINT           (system)
 ROWKEY         | VARCHAR(STRING)  (system)
 DATE           | BIGINT
 FACILITY       | INTEGER
 HOST           | VARCHAR(STRING)
 LEVEL          | INTEGER
 MESSAGE        | VARCHAR(STRING)
 CHARSET        | VARCHAR(STRING)
 REMOTE_ADDRESS | VARCHAR(STRING)
 HOSTNAME       | VARCHAR(STRING)
--------------------------------------------

Queries that write into this STREAM
-----------------------------------
id:CSAS_SYSLOG_INVALID_USER_LOGIN - CREATE STREAM SYSLOG_INVALID_USER_LOGIN AS SELECT * FROM SYSLOG WHERE HOST='rpi-03' AND MESSAGE LIKE '%Invalid user%';

For query topology and execution plan please run: EXPLAIN <QueryId>

Local runtime statistics
------------------------
messages-per-sec:     13.46   total-messages:      1335     last-message: 3/12/18 1:59:35 PM GMT
 failed-messages:         0 failed-messages-per-sec:         0      last-failed:       n/a
(Statistics of the local KSQL server interaction with the Kafka topic SYSLOG_INVALID_USER_LOGIN)
ksql>

Summary & What’s Next…

KSQL gives anyone the power to write stream processing applications, using the simple declarative language of SQL. In this article we’ve seen how we can inspect inbound streams of syslog data and easily create a live stream of filtered messages written to a second Kafka topic.

Diagram showing KSQL filtering inbound syslog data

 

In the next article we’ll take a brief detour from KSQL itself, and look at using Python to write a very simple push-notification system, driven by events arriving in a Kafka topic. We’ll see some simple anomaly detection, building on KSQL’s stateful aggregation capabilities.

After that we’ll return to KSQL in force, using its superpowers to join our inbound syslog data with useful data from a datastore that we can use to enrich the syslog data in realtime.

To read more about building streaming data pipelines with Apache Kafka and KSQL, check out the following articles:

The post We ❤️ syslogs: Real-time syslog Processing with Apache Kafka and KSQL—Part 1: Filtering appeared first on Confluent.

Viewing all 175 articles
Browse latest View live