From 76e1fd52e549f6751dbb0b2e99f43701631bc0aa Mon Sep 17 00:00:00 2001 From: Pawel Garbacki Date: Mon, 28 Apr 2014 23:14:58 -0700 Subject: [PATCH] Initial checkin udpated --- DESIGN.md | 205 ++++++++++++ README.md | 102 +++++- pom.xml | 298 ++++++++++++++++++ src/main/assembly/secor.xml | 40 +++ src/main/config/kafka.test.properties | 110 +++++++ src/main/config/log4j.dev.properties | 18 ++ src/main/config/log4j.prod.properties | 18 ++ src/main/config/secor.common.properties | 72 +++++ src/main/config/secor.dev.backup.properties | 32 ++ .../config/secor.dev.partition.properties | 32 ++ src/main/config/secor.dev.properties | 24 ++ src/main/config/secor.prod.backup.properties | 32 ++ .../config/secor.prod.partition.properties | 32 ++ src/main/config/secor.prod.properties | 41 +++ src/main/config/zookeeper.test.properties | 20 ++ .../pinterest/secor/common/FileRegistry.java | 232 ++++++++++++++ .../pinterest/secor/common/KafkaClient.java | 187 +++++++++++ .../pinterest/secor/common/LogFilePath.java | 204 ++++++++++++ .../pinterest/secor/common/OffsetTracker.java | 91 ++++++ .../secor/common/OstrichAdminService.java | 71 +++++ .../pinterest/secor/common/SecorConfig.java | 163 ++++++++++ .../secor/common/TopicPartition.java | 68 ++++ .../secor/common/ZookeeperConnector.java | 182 +++++++++++ .../pinterest/secor/consumer/Consumer.java | 117 +++++++ .../pinterest/secor/main/ConsumerMain.java | 80 +++++ .../secor/main/LogFilePrinterMain.java | 70 ++++ .../secor/main/LogFileVerifierMain.java | 103 ++++++ .../secor/main/PartitionFinalizerMain.java | 51 +++ .../secor/main/ProgressMonitorMain.java | 49 +++ .../main/TestLogMessageProducerMain.java | 80 +++++ .../secor/main/ZookeeperClientMain.java | 91 ++++++ .../com/pinterest/secor/message/Message.java | 74 +++++ .../secor/message/ParsedMessage.java | 46 +++ .../pinterest/secor/parser/MessageParser.java | 46 +++ .../secor/parser/OffsetMessageParser.java | 40 +++ .../secor/parser/PartitionFinalizer.java | 212 +++++++++++++ .../pinterest/secor/parser/QuboleClient.java | 112 +++++++ .../secor/parser/ThriftMessageParser.java | 83 +++++ .../pinterest/secor/reader/MessageReader.java | 133 ++++++++ .../pinterest/secor/tools/LogFilePrinter.java | 63 ++++ .../secor/tools/LogFileVerifier.java | 208 ++++++++++++ .../secor/tools/ProgressMonitor.java | 157 +++++++++ .../secor/tools/RandomPartitioner.java | 35 ++ .../secor/tools/TestLogMessageProducer.java | 76 +++++ .../pinterest/secor/uploader/Uploader.java | 199 ++++++++++++ .../com/pinterest/secor/util/FileUtil.java | 114 +++++++ .../java/com/pinterest/secor/util/IdUtil.java | 41 +++ .../pinterest/secor/util/RateLimitUtil.java | 41 +++ .../pinterest/secor/util/ReflectionUtil.java | 45 +++ .../com/pinterest/secor/util/StatsUtil.java | 38 +++ .../pinterest/secor/writer/MessageWriter.java | 80 +++++ src/main/scripts/run_consumer.sh | 30 ++ src/main/scripts/run_kafka_class.sh | 65 ++++ src/main/scripts/run_tests.sh | 217 +++++++++++++ src/main/scripts/run_zookeeper_command.sh | 25 ++ src/main/thrift/secor.thrift | 15 + .../secor/common/FileRegistryTest.java | 158 ++++++++++ .../secor/common/LogFilePathTest.java | 86 +++++ .../secor/uploader/UploaderTest.java | 187 +++++++++++ 59 files changed, 5538 insertions(+), 3 deletions(-) create mode 100644 DESIGN.md create mode 100644 pom.xml create mode 100644 src/main/assembly/secor.xml create mode 100644 src/main/config/kafka.test.properties create mode 100644 src/main/config/log4j.dev.properties create mode 100644 src/main/config/log4j.prod.properties create mode 100644 src/main/config/secor.common.properties create mode 100644 src/main/config/secor.dev.backup.properties create mode 100644 src/main/config/secor.dev.partition.properties create mode 100644 src/main/config/secor.dev.properties create mode 100644 src/main/config/secor.prod.backup.properties create mode 100644 src/main/config/secor.prod.partition.properties create mode 100644 src/main/config/secor.prod.properties create mode 100644 src/main/config/zookeeper.test.properties create mode 100644 src/main/java/com/pinterest/secor/common/FileRegistry.java create mode 100644 src/main/java/com/pinterest/secor/common/KafkaClient.java create mode 100644 src/main/java/com/pinterest/secor/common/LogFilePath.java create mode 100644 src/main/java/com/pinterest/secor/common/OffsetTracker.java create mode 100644 src/main/java/com/pinterest/secor/common/OstrichAdminService.java create mode 100644 src/main/java/com/pinterest/secor/common/SecorConfig.java create mode 100644 src/main/java/com/pinterest/secor/common/TopicPartition.java create mode 100644 src/main/java/com/pinterest/secor/common/ZookeeperConnector.java create mode 100644 src/main/java/com/pinterest/secor/consumer/Consumer.java create mode 100644 src/main/java/com/pinterest/secor/main/ConsumerMain.java create mode 100644 src/main/java/com/pinterest/secor/main/LogFilePrinterMain.java create mode 100644 src/main/java/com/pinterest/secor/main/LogFileVerifierMain.java create mode 100644 src/main/java/com/pinterest/secor/main/PartitionFinalizerMain.java create mode 100644 src/main/java/com/pinterest/secor/main/ProgressMonitorMain.java create mode 100644 src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java create mode 100644 src/main/java/com/pinterest/secor/main/ZookeeperClientMain.java create mode 100644 src/main/java/com/pinterest/secor/message/Message.java create mode 100644 src/main/java/com/pinterest/secor/message/ParsedMessage.java create mode 100644 src/main/java/com/pinterest/secor/parser/MessageParser.java create mode 100644 src/main/java/com/pinterest/secor/parser/OffsetMessageParser.java create mode 100644 src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java create mode 100644 src/main/java/com/pinterest/secor/parser/QuboleClient.java create mode 100644 src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java create mode 100644 src/main/java/com/pinterest/secor/reader/MessageReader.java create mode 100644 src/main/java/com/pinterest/secor/tools/LogFilePrinter.java create mode 100644 src/main/java/com/pinterest/secor/tools/LogFileVerifier.java create mode 100644 src/main/java/com/pinterest/secor/tools/ProgressMonitor.java create mode 100644 src/main/java/com/pinterest/secor/tools/RandomPartitioner.java create mode 100644 src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java create mode 100644 src/main/java/com/pinterest/secor/uploader/Uploader.java create mode 100644 src/main/java/com/pinterest/secor/util/FileUtil.java create mode 100644 src/main/java/com/pinterest/secor/util/IdUtil.java create mode 100644 src/main/java/com/pinterest/secor/util/RateLimitUtil.java create mode 100644 src/main/java/com/pinterest/secor/util/ReflectionUtil.java create mode 100644 src/main/java/com/pinterest/secor/util/StatsUtil.java create mode 100644 src/main/java/com/pinterest/secor/writer/MessageWriter.java create mode 100755 src/main/scripts/run_consumer.sh create mode 100755 src/main/scripts/run_kafka_class.sh create mode 100755 src/main/scripts/run_tests.sh create mode 100755 src/main/scripts/run_zookeeper_command.sh create mode 100644 src/main/thrift/secor.thrift create mode 100644 src/test/java/com/pinterest/secor/common/FileRegistryTest.java create mode 100644 src/test/java/com/pinterest/secor/common/LogFilePathTest.java create mode 100644 src/test/java/com/pinterest/secor/uploader/UploaderTest.java diff --git a/DESIGN.md b/DESIGN.md new file mode 100644 index 000000000..3062bd14a --- /dev/null +++ b/DESIGN.md @@ -0,0 +1,205 @@ +# Secor: kafka message persistence layer + +## Prerequisites + +This document assumes familiarity with [Apache Kafka]. + +## Objectives + +*Secor* is a service persisting [Kafka] logs to [Amazon S3]. Its design is motivated by the following needs: + +* **minimized chance of data loss and corruption:** this is by far the highest priority objective. Logs are the basis of billing and as such they cannot lie, + +* **protection against bad code****:** data corruption caused by bugs in the message parsers should be reversible, + +* **low log-to-ready_for_consumption delay:** logged messages should be ready for consumption by analytical tools asap. We don’t exclude a near-real-time use case in the near future, + +* **horizontal scalability:** the system should be able to outgrow its initial use case. The scalability should be achieved by adding more replicas of components rather than resources per component, + +* **extensibility:** it should be easy to add new component types (e.g., log formats, message parsers and filters), + +* **design clarity (and simplicity):** simple solutions are easy to understand, implement, maintain, and fix, + +* **minimized incident-to-alert delay:** incidents are an unavoidable reality of working with distributed systems, no matter how reliable. A prompt alerting solution should be an integral part of the design, + +* **zero downtime upgrades:** it should be possible to upgrade the system to a new version in a way transparent to the downstream data clients, + +* **dependence on public APIs:** the system should reply on public [Kafka] APIs only. Furthermore, it should be compatible with the most recent [Kafka] version (0.8) which offers significant improvements over 0.7, and it comes with Go language bindings (required by other pieces of the Ads infra). + +No-goals: + +* **minimized resource footprint:** this may become an important objective at some point but currently we don’t optimize for machine or storage footprint. + +Secor will be initially used to persist Ads impression logs but in the future it may be considered as a replacement of the current logging pipeline. + +## Related work + +There is a number of open source [Kafka] consumers saving data to [S3]. To the best of our knowledge, none of them is + +* able to auto-detect new topics and kafka partitions, + +* dynamically distribute the load across consumers at runtime, + +* control the upload policy on a per-topic-kafka-partition basis (it requires the ability to commit message offsets at the granularity of an individual topic/kafka partition), + +* plug in custom message parsing logic. + +## Design overview + +Secor is a distributed [Kafka] *consumer*. Individual consumers are divided into *groups*. For every partition (of a [Kafka] topic) there is exactly one consumer in each group processing messages posted to that partition. You may think of a group as an application implementing a specific message filtering and/or conversion logic. + +In the initial implementation, there are two groups, the backup group, and the partition group. The *backup* group simply persists all messages on [S3] in the format they are received by [Kafka]. It does not look into message content. The *partition* group parses each message to extract application-defined partition names. Messages are then grouped and stored in a directory structure arranged according to those names. The name "partition" is a bit overloaded as it is used to describe both the numeric Kafka topic partition as well as an arbitrary string derived from the message content. To disambiguate those two, while talking about the former we will always explicitly call it *Kafka partition*. + +## Backup group + +The purpose of the backup group is to create a copy of the messages in the exact format they are received by the [Kafka] brokers. Consumers in this group are designed to be dead-simple, performant, and highly resilient. Since the data copied verbatim, no code upgrades are required to support new message types. + +In most of the cases, the output of the copy backup is not suitable for direct consumption by data analysis tools. The existence of that group increases confidence levels and allows raw data to be reprocessed for backfilling of outputs that were messed up due to bugs in other, logically and computationally more involving groups. For instance, if a bug was introduced in the message parsing logic implemented by the partition group, the processed data may be incorrect. If that bug is subtle, noticing it may take a while. [Kafka] brokers store messages only for a limited time (usually a few days). If the original log data is lost, there is no way retroactively fix corrupted output. The backup group makes sure that no data is ever lost. + +## Partition group + +Partition group clusters log messages based on custom logic parametrized with message content. More precisely, a consumer in this group parses the message content to extract a list of partitions and places the message in a file with other messages that belong to those partitions. In the majority of cases, the message partition list contains a single value, the date corresponding to the timestamp recorded in the message. + +The main complication in the implementation of consumers in this group is the fact that different topics may use different message formats (e.g., json vs thrift). Furthermore, extracting partitions requires the knowledge of the exact message schema (e.g., the name of the json or thrift field carrying the timestamp). Those two properties introduce the need for specialized parsers. Furthermore, since message formats may change, we need a mechanism to keep parser configs up to date. Versioning and version rewinding for retroactive backfills adds another level of complexity. See the section on upgrades for more details. + +## Consumer + +A consumer is a component responsible for reading messages submitted to [Kafka] topics, optionally transforming them, and writing the output to [S3]. A consumer belongs to exactly one group but it may process messages from multiple topics and [Kafka] partitions. [Kafka] guarantees that every message is delivered to exactly one consumer in a group. + +[S3] does not support appending to existing files so messages need to be stored locally first before being uploaded (in batches) to [S3]. Neither [S3] nor data processing tools like [Hive] enjoy working with a large number of small files. Deciding when to upload a local file to [S3] is a matter of *upload policy*. We envision two types of policy rules: time-based and size-based. The time-based rule forces an upload if the time delta between the current time and file creation time is greater than the predefined threshold. The size-based rule forces uploads of files larger than a given size. A file is removed from local disk immediately after its upload to [S3]. At that time we also record the offset of the latest message in the uploaded file in Zookeeper. + +In addition to a upload policy, a consumer is parametrized with a log reader, a message parser, log writer, and file uploader. + +The *reader* reads the next message. In most of the cases, messages are retrieved from Kafka but it is possible to use alternative message sources. E.g., for the purpose of backfilling, we could implement a reader streaming messages from a file. + +The *message parser* is responsible for extracting the payload and the list of partitions from a message. + +The *writer* appends the message to a local file corresponding to the message’s topic and partitions. For a given combination of topic and partitions there is exactly one file appended to at a given time. + +The *uploader* moves log files from local directory to [S3] and commits the last uploaded offsets to zookeeper. + +### Offset management + +A Secor consumer is built on top of [Kafka high level consumer API](https://kafka.apache.org/documentation.html#highlevelconsumerapi). The offset of the last committed (persisted in [S3]) message for each topic/Kafka partition pair is stored in zookeeper. Certain types of events such as join of a new consumer or crash of an existing one trigger so called *rebalance*. Rebalance may change the allocation of topic/Kafka partitions to consumers. A topic/Kafka partition that changes the ownership will be consumed from the last committed offset + 1 by the new owner (note: offsets increase by one between messages). The greatest challenge in the design of the consumer logic is to handle rebalance events while guaranteeing that each message is persisted (in [S3]) exactly once. Below we outline an algorithm implementing a logic with this property. + +#### Data structures + +Every consumer keeps the following information locally. + +`last_seen_offset`: a mapping from `` to the greatest offset of a message seen (but not necessarily committed) in that topic/Kafka partition. + +`last_committed_offset`: a mapping from `` to the greatest committed offset of a message in that topic/Kafka partition. Committed offset stored locally does not have to agree with the value stored currently in zookeeper but it corresponds to a value that zookeeper had at some point in the past. + +#### Message consumption algorithm + +For simplicity, the following pseudocode ignores all details related to message parsing. + +``` +consumer.run() { + do forever { + message = reader.read(); + writer.write(message); + uploader.check_policy(); + } +} +``` + +``` +reader.read() { + message = kafka.next(); + t = message.topic; + p = message.partition; + if message.offset <= last_committed_offset[t, p] { + return null; + } + return message; +} +``` + +``` +writer.write(message) { + t = message.topic; + p = message.partition; + if message.offset != last_seen_offset[t, p] + 1 { + delete_local_files_for_topic_partition(t, p); + } + write_to_local_file(message); + last_seen_offset[t, p] = message.offset; +} +``` + +``` +uploader.check_policy() { + for (t, p) in owned_topic_partitions { + // Policy example: + // sum(file.size) > max_size or + // max(file.mtime) < now - max_age + if policy.should_upload(t, p) { + old_offset = last_committed_offset[t, p]; + new_offset = zookeeper.get_last_committed_offset(t, p); + last_committed_offset[t, p] = new_offset; + if old_offset == new_offset { + // Exclusive lock: prevent any modifications of the committed + // offset for a given topic/kafka partition. + zookeeper.lock(t, p); + upload_files_to_s3(t, p); + zookeeper.set_last_committed_offset(t, p, seen_offset[t, p]); + zookeeper.unlock(); + } + + // Rewrite local files for a given topic and partition removing + // messages with offsets lower than or equal to the given + // offset. + trim_files(t, p, new_offset); + } + } +} +``` + +## Output file names + +The output of consumers is stored on local (or EBS) disks first and eventually uploaded to s3. The local and s3 file name format follows the same pattern. Directory paths track topic and partition names. File basename contains the Kafka partition number and the Kafka offset of the first message in that file. Additionally, files are labeled with generation count. Generation is basically a version number of the Secor software that increments between non-compatible releases. Generations allow us to separate outputs of Secor versions during testing, rolling upgrades, etc. The consumer group is not included explicitly in the output path. We expect that the output of different consumer groups will go to different top-level directories. + +Putting this all together, a message with timestamp `` written to topic ``, Kafka partition `` at offset `` by software with generation `` will end up in file `s3://logs///__.seq` where `` <= ``. + +The nice property of the proposed file format is that given a list of output files and a Kafka message, we can tell which file contains the output for that message. In other words, we can track correspondence between the output files of different consumer groups. For instance, assume that a bug in the code resulted in logs for a given date being incorrectly processed. We now need to remove all output files produced by the partition group and regenerate them from the files written by the backup group. The composition of file paths guarantees that we can tell which backup files contain the relevant raw records from the names of the removed partition group output files. + +## New consumer code rollouts + +The upgrade procedure is as simple as killing consumers running the old version of the code and letting them pick up new binaries upon restart. Generation numbers provide output isolation across incompatible releases. + +## Failure scenarios + +### Consumer crash + +1. A consumer process dies unexpectedly. + +2. [Kafka] notices consumer departure and assigns its topics/partitions to someone else during the rebalancing process. + +3. A new consumer is assigned to a topic/partition previously handled by the failed consumer. + +4. The new consumer starts processing messages from the last committed offset recorded in Zookeeper. + +5. The failed consumer comes back (recovers), say on the same machine where it was running before. + +6. The recovered consumer discards (i.e., removes) all local log files. + +7. Kafka assigns the recovered consumer topics/partitions. + +### Consumers cannot keep up with logging rate + +To increase the processing capacity on the consumer end we can simply start more consumer processes. Kafka will automatically rebalance the topic/partition assignments across the population of all available consumers. + +### Parser bug + +Assume that a parser bug caused misformatted output being written to s3 starting with [Kafka] message offset . The output file name format allows us to easily identify misformatted files to remove. If is still in kafka, we can simply reset topic/partition offsets in Zookeeper to that value. Otherwise, we can write a simple MR applying the exactly same parsing logic and upload policy as the consumer. The nice thing about this approach is that backfilling can be done in parallel with the ongoing message processing. I.e., there is no need to take the consumers down while fixing corrupted output. + +### Zookeeper failing + +Secor won't be able to commit offsets while zk is down. As soon as zk comes back, Secor will resume commits. + +[Apache Kafka]:http://kafka.apache.org/ +[Kafka]:http://kafka.apache.org/ +[Amazon S3]:http://aws.amazon.com/s3/ +[S3]:http://aws.amazon.com/s3/ +[Hive]:http://hive.apache.org/ diff --git a/README.md b/README.md index e602bb599..9f2afbb9b 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,100 @@ -secor -===== +# Secor + +Secor is a service persisting [Kafka] logs to [Amazon S3]. + +## Key features + - **strong consistency**: as long as [Kafka] is not dropping messages (e.g., due to aggresive cleanup policy) before Secor is able to read them, it is guaranteed that each message will be saved in exacly one [S3] file. This property is not compromized by the notorious temporal inconsisteny of [S3] caused by the [eventual consistency] model, + - **fault tolerance**: any component of Secor is allowed to crash at any given point without compromising data integrity, + - **load distribution**: Secor may be distributed across multiple machines, + - **horizontal scalability**: scaling the system out to handle more load is as easy as starting extra Secor processes. Reducing the resource footprint can be achieved by killing any of the running Secor processes. Neither ramping up nor down has any impact on data consistency, + - **output partitioning**: Secor parses incoming messages and puts them under partitioned s3 paths to enable direct import into systems like [Hive], + - **configurable upload policies**: commit points controlling when data is persisted in S3 are configured through size-based and time-based policies (e.g., upload data when local buffer reaches size of 100MB and at least once per hour), + - **monitoring**: metrics tracking various performace properties are exposed through [Ostrich] and optionaly exported to [OpenTSDB], + - **customizability**: external log message parser may be loaded by updating the configuration, + - **Qubole interface**: Secor connects to [Qubole] to add finalized output partitions to Hive tables. + +## Setup Guide + +##### Get Secor code +```sh +git clone [git-repo-url] secor +cd secor +``` + +##### Customize configuration parameters +Edit `src/main/config/*.properties` files to specify parameters describing the environment. Those files contain comments describing the meaning of individual parameters. + +##### Create and install jars +```sh +mvn package +mkdir ${SECOR_INSTALL_DIR} # directory to place Secor binaries in. +tar -zxvf target/secor-0.1-SNAPSHOT-bin.tar.gz -C ${SECOR_INSTALL_DIR} +``` + +##### Run tests (optional) +```sh +cd ${SECOR_INSTALL_DIR} +./scripts/run_tests.sh +``` + +##### Run Secor +```sh +cd ${SECOR_INSTALL_DIR} +java -ea -Dsecor_group=secor_partition -Dlog4j.configuration=log4j.prod.properties -Dconfig=secor.prod.backup.properties -cp secor-0.1-SNAPSHOT.jar:lib/* com.pinterest.secor.main.ConsumerMain +``` + +## Tools +Secor comes with a number of tools impelementing interactions with the environment. + +##### Log file printer +Log file printer displays the content of a log file. + +```sh +java -ea -Dlog4j.configuration=log4j.prod.properties -Dconfig=secor.prod.backup.properties -cp "secor-0.1-SNAPSHOT.jar:lib/*" com.pinterest.secor.main.LogFilePrinterMain -f s3n://bucket/path +``` + +##### Log file verifier +Log file verifier checks the consistency of log files. + +```sh +java -ea -Dlog4j.configuration=log4j.prod.properties -Dconfig=secor.prod.backup.properties -cp "secor-0.1-SNAPSHOT.jar:lib/*" com.pinterest.secor.main.LogFileVerifierMain -t topic -q +``` + +##### Partition finalizer +Topic finalizer writes _SUCCESS files to date partitions that very likely won't be receiving any new messages and (optionaly) adds the corresponding dates to [Hive] through [Qubole] API. + +```sh +java -ea -Dlog4j.configuration=log4j.prod.properties -Dconfig=secor.prod.backup.propertie -cp "secor-0.1-SNAPSHOT.jar:lib/*" com.pinterest.secor.main.PartitionFinalizerMain +``` + +##### Progress monitor +Progress monitor exports offset consumption lags per topic partition to [OpenTSDB]. Lags track how far Secor is behind the producers. + +```sh +java -ea -Dlog4j.configuration=log4j.prod.properties -Dconfig=secor.prod.backup.properties -cp "secor-0.1-SNAPSHOT.jar:lib/*" com.pinterest.secor.main.ProgressMonitorMain +``` + +## Detailed design + +Design details are available in [DESIGN.md](DESIGN.md). + +## License + +Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/licenses/LICENSE-2.0.html). + +## Maintainers +Pawel Garbacki + +## Help + +If you have any questions or comments, you can reach us at [secor-users@googlegroups.com](secor-users@googlegroups.com) + +[Kafka]:http://kafka.apache.org/ +[Amazon S3]:http://aws.amazon.com/s3/ +[S3]:http://aws.amazon.com/s3/ +[eventual consistency]:http://docs.aws.amazon.com/AmazonS3/latest/dev/Introduction.html#ConsistencyMode +[Hive]:http://hive.apache.org/ +[Ostrich]: https://github.com/twitter/ostrich +[OpenTSDB]: http://opentsdb.net/ +[Qubole]: http://www.qubole.com/ -Secor is a service implementing Kafka log persistance diff --git a/pom.xml b/pom.xml new file mode 100644 index 000000000..5d2ea1d3a --- /dev/null +++ b/pom.xml @@ -0,0 +1,298 @@ + + + 4.0.0 + + com.pinterest.secor + secor + 0.1-SNAPSHOT + jar + Kafka to s3 logs exporter + + + + Apache License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + A business-friendly OSS license + + + + + 1.6 + 1.6 + UTF-8 + UTF-8 + + + + + Twitter public Maven repo + http://maven.twttr.com + + + Typesafe public Maven repo + http://repo.typesafe.com/typesafe/releases + + + + + Twitter public Maven repo + http://maven.twttr.com + + + Typesafe public Maven repo + http://repo.typesafe.com/typesafe/releases + + + + + + org.apache.kafka + kafka_2.10 + 0.8.0 + + + org.slf4j + slf4j-simple + + + + + log4j + log4j + 1.2.15 + + + com.sun.jmx + jmxri + + + com.sun.jdmk + jmxtools + + + javax.jms + jms + + + + + org.slf4j + slf4j-api + 1.7.2 + + + org.slf4j + jcl-over-slf4j + 1.7.2 + + + org.slf4j + slf4j-log4j12 + 1.7.2 + + + commons-configuration + commons-configuration + 1.9 + + + org.apache.hadoop + hadoop-core + 1.2.1 + + + org.apache.thrift + libthrift + 0.5.0 + + + com.twitter + ostrich_2.10 + 9.2.1 + + + com.twitter.common.zookeeper + lock + 0.0.7 + + + com.google.guava + guava + 14.0 + + + net.minidev + json-smart + 1.0.9 + + + junit + junit + 4.11 + test + + + org.mockito + mockito-all + 1.9.5 + + + org.powermock + powermock-module-junit4 + 1.5.2 + test + + + org.powermock + powermock-api-mockito + 1.5.2 + test + + + + + + + src/main/config + + + + + org.codehaus.mojo + exec-maven-plugin + 1.1 + + + compile + + exec + + + git + + log + --pretty=format:build_revision=%H + -n1 + + target/build.properties + + + + + + + maven-dependency-plugin + + + package + + copy-dependencies + + + ${project.build.directory}/lib + + + + + + org.codehaus.mojo + exec-maven-plugin + 1.1 + + + compile + + exec + + + git + + log + --pretty=format:build_revision=%H + -n1 + + target/classes/com/pinterest/secor/common/build.properties + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + create-distribution + package + + single + + + + src/main/assembly/secor.xml + + + + + + + + com.twitter + maven-finagle-thrift-plugin + 0.0.9 + + + thrift-sources + generate-sources + + compile + + + + finagle + + + finagle + + + + + thrift-test-sources + generate-test-sources + + testCompile + + + + finagle + + + finagle + + + + + + + finagle + + + + + org.codehaus.mojo + license-maven-plugin + 1.6 + + + download-licenses + + download-licenses + + + + + + + \ No newline at end of file diff --git a/src/main/assembly/secor.xml b/src/main/assembly/secor.xml new file mode 100644 index 000000000..099ea3bd1 --- /dev/null +++ b/src/main/assembly/secor.xml @@ -0,0 +1,40 @@ + + + bin + + + tar.gz + + + + src/main/scripts + scripts + + *.sh + + + + target + + + secor*.jar + + + + target/classes + + + *.properties + + + + + + + lib + true + false + runtime + + + diff --git a/src/main/config/kafka.test.properties b/src/main/config/kafka.test.properties new file mode 100644 index 000000000..e79bbc8a5 --- /dev/null +++ b/src/main/config/kafka.test.properties @@ -0,0 +1,110 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id=0 + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port=9092 + +# Hostname the broker will bind to and advertise to producers and consumers. +# If not set, the server will bind to all interfaces and advertise the value returned from +# from java.net.InetAddress.getCanonicalHostName(). +#host.name=localhost + +# The number of threads handling network requests +num.network.threads=2 + +# The number of threads doing disk I/O +num.io.threads=2 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=1048576 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=1048576 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# A comma seperated list of directories under which to store log files +log.dirs=/tmp/secor_dev/kafka-logs + +# The number of logical partitions per topic per server. More partitions allow greater parallelism +# for consumption, but also mean more files. +num.partitions=2 + +############################# Log Flush Policy ############################# + +# The following configurations control the flush of data to disk. This is among the most +# important performance knob in kafka. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data may be lost if you are not using replication. +# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +log.flush.interval.ms=1000 + +# Per-topic overrides for log.flush.interval.ms +#log.flush.intervals.ms.per.topic=topic1:1000, topic2:3000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +log.segment.bytes=536870912 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.cleanup.interval.mins=1 + +############################# Zookeeper ############################# + +# Zookeeper connection string (see zookeeper docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect=localhost:2181 + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=1000000 + + diff --git a/src/main/config/log4j.dev.properties b/src/main/config/log4j.dev.properties new file mode 100644 index 000000000..720a7907d --- /dev/null +++ b/src/main/config/log4j.dev.properties @@ -0,0 +1,18 @@ +# log4j logging dev configuration. + +# root logger. +log4j.rootLogger=DEBUG, CONSOLE, ROLLINGFILE + +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +log4j.appender.CONSOLE.Threshold=INFO +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} [%t] (%C:%L) %-5p %m%n + +log4j.appender.ROLLINGFILE=org.apache.log4j.RollingFileAppender +log4j.appender.ROLLINGFILE.Threshold=DEBUG +log4j.appender.ROLLINGFILE.File=/tmp/secor_dev/logs/secor.log +# keep log files up to 1G +log4j.appender.ROLLINGFILE.MaxFileSize=20MB +log4j.appender.ROLLINGFILE.MaxBackupIndex=50 +log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout +log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} [%t] (%C:%L) %-5p %m%n diff --git a/src/main/config/log4j.prod.properties b/src/main/config/log4j.prod.properties new file mode 100644 index 000000000..298e2cdb9 --- /dev/null +++ b/src/main/config/log4j.prod.properties @@ -0,0 +1,18 @@ +# log4j logging configuration. + +# root logger. +log4j.rootLogger=DEBUG, CONSOLE, ROLLINGFILE + +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +log4j.appender.CONSOLE.Threshold=WARN +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} [%t] (%C:%L) %-5p %m%n + +log4j.appender.ROLLINGFILE=org.apache.log4j.RollingFileAppender +log4j.appender.ROLLINGFILE.Threshold=INFO +log4j.appender.ROLLINGFILE.File=/mnt/secor_data/logs/secor-${secor_group}.log +# keep log files up to 1G +log4j.appender.ROLLINGFILE.MaxFileSize=20MB +log4j.appender.ROLLINGFILE.MaxBackupIndex=50 +log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout +log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} [%t] (%C:%L) %-5p %m%n diff --git a/src/main/config/secor.common.properties b/src/main/config/secor.common.properties new file mode 100644 index 000000000..4100c96ca --- /dev/null +++ b/src/main/config/secor.common.properties @@ -0,0 +1,72 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +############ +# MUST SET # +############ + +# Regular expression matching names of consumed topics. +secor.kafka.topic_filter=.* + +# AWS authentication credentials. +aws.access.key= +aws.secret.key= + +################ +# END MUST SET # +################ + +# Zookeeper config. +zookeeper.session.timeout.ms=3000 +zookeeper.sync.time.ms=200 + +# Impacts how frequently the upload logic is triggered if no messages are delivered. +kafka.consumer.timeout.ms=10000 + +# Port of the broker serving topic partition metadata. +kafka.seed.broker.port=9092 + +# Secor generation is a version that should be incremented during non-backwards-compabile +# Secor releases. Generation number is one of the components of generated log file names. +# Generation number makes sure that outputs of different Secor versions are isolated. +secor.generation=1 + +# Number of consumer threads per Secor process. +secor.consumer.threads=7 + +# Consumption rate limit enforced at the process level (not a consumer-thread level). +secor.messages.per.second=10000 + +# Used by the "backup" consumer group only. +# Number of continous message offsets that constitute a single offset= partition on s3. +# Example: +# if set to 10, +# messages with offsets 0 to 9 will be written to s3 path s3n://.../offset=0/... +# messages with offsets 10 to 19 will be written to s3 path s3n://.../offset=10/... +# ... +secor.offsets.per.partition=10000000 + +# How long does it take for secor to forget a topic partition. Applies to stats generation only. +secor.topic_partition.forget.seconds=600 + +# Secor comes with a tool that adds Hive partitions for finalized topics. Currently, we support +# only Hive clusters accessible through Qubole. The token gives access to the Qubole API. +# It is available at https://api.qubole.com/users/edit +qubole.api.token= + +# Secor can export stats such as consumption lag (in seconds and offsets) per topic partition. +# Leave empty to disable this functionality. +tsdb.hostport= + diff --git a/src/main/config/secor.dev.backup.properties b/src/main/config/secor.dev.backup.properties new file mode 100644 index 000000000..fac8845d5 --- /dev/null +++ b/src/main/config/secor.dev.backup.properties @@ -0,0 +1,32 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +include=secor.dev.properties + +# Port of the Ostrich server. +secor.kafka.group=secor_backup + +# Parser class that extracts s3 partitions from consumed messages. +secor.message.parser.class=com.pinterest.secor.parser.OffsetMessageParser + +# S3 path where sequence files are stored. +secor.s3.path=secor_dev/backup + +# Local path where sequence files are stored before they are uploaded to s3. +secor.local.path=/tmp/secor_dev/message_logs/backup + +# Port of the Ostrich server. +ostrich.port=9999 + diff --git a/src/main/config/secor.dev.partition.properties b/src/main/config/secor.dev.partition.properties new file mode 100644 index 000000000..d029c86d0 --- /dev/null +++ b/src/main/config/secor.dev.partition.properties @@ -0,0 +1,32 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +include=secor.dev.properties + +# Name of the Kafka consumer group. +secor.kafka.group=secor_partition + +# Parser class that extracts s3 partitions from consumed messages. +secor.message.parser.class=com.pinterest.secor.parser.ThriftMessageParser + +# S3 path where sequence files are stored. +secor.s3.path=secor_dev/partition + +# Local path where sequence files are stored before they are uploaded to s3. +secor.local.path=/tmp/secor_dev/message_logs/partition + +# Port of the Ostrich server. +ostrich.port=9998 + diff --git a/src/main/config/secor.dev.properties b/src/main/config/secor.dev.properties new file mode 100644 index 000000000..b1ad32099 --- /dev/null +++ b/src/main/config/secor.dev.properties @@ -0,0 +1,24 @@ +include=secor.common.properties + +############ +# MUST SET # +############ + +# Name of the s3 bucket where log files are stored. +secor.s3.bucket= + +################ +# END MUST SET # +################ + +kafka.seed.broker.host=localhost +kafka.seed.broker.port=9092 + +zookeeper.quorum=localhost:2181 + +# Upload policies. +# 10K +secor.max.file.size.bytes=10000 +# 1 minute +secor.max.file.age.seconds=60 + diff --git a/src/main/config/secor.prod.backup.properties b/src/main/config/secor.prod.backup.properties new file mode 100644 index 000000000..35f57f0e8 --- /dev/null +++ b/src/main/config/secor.prod.backup.properties @@ -0,0 +1,32 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +include=secor.prod.properties + +# Name of the Kafka consumer group. +secor.kafka.group=secor_backup + +# Parser class that extracts s3 partitions from consumed messages. +secor.message.parser.class=com.pinterest.secor.parser.OffsetMessageParser + +# S3 path where sequence files are stored. +secor.s3.path=raw_logs/secor_backup + +# Local path where sequence files are stored before they are uploaded to s3. +secor.local.path=/mnt/secor_data/message_logs/backup + +# Port of the Ostrich server. +ostrich.port=9999 + diff --git a/src/main/config/secor.prod.partition.properties b/src/main/config/secor.prod.partition.properties new file mode 100644 index 000000000..14e36e667 --- /dev/null +++ b/src/main/config/secor.prod.partition.properties @@ -0,0 +1,32 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +include=secor.prod.properties + +# Name of the Kafka consumer group. +secor.kafka.group=secor_partition + +# Parser class that extracts s3 partitions from consumed messages. +secor.message.parser.class=com.pinterest.secor.parser.ThriftMessageParser + +# S3 path where sequence files are stored. +secor.s3.path=raw_logs + +# Local path where sequence files are stored before they are uploaded to s3. +secor.local.path=/mnt/secor_data/message_logs/partition + +# Port of the Ostrich server. +ostrich.port=9998 + diff --git a/src/main/config/secor.prod.properties b/src/main/config/secor.prod.properties new file mode 100644 index 000000000..0801a5c99 --- /dev/null +++ b/src/main/config/secor.prod.properties @@ -0,0 +1,41 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +include=secor.common.properties + +############ +# MUST SET # +############ + +# Name of the s3 bucket where log files are stored. +secor.s3.bucket= + +# Name of one (random) Kafka broker host that is used to retrieve metadata. +# TODO(pawel): use a list of nodes or even better, extract active brokers from zookeeper. +kafka.seed.broker.host= + +# List of Kafka Zookeeper servers. +zookeeper.quorum= + +################ +# END MUST SET # +################ + +# Upload policies. +# 200MB +secor.max.file.size.bytes=200000000 +# 1 hour +secor.max.file.age.seconds=3600 + diff --git a/src/main/config/zookeeper.test.properties b/src/main/config/zookeeper.test.properties new file mode 100644 index 000000000..f71adadd8 --- /dev/null +++ b/src/main/config/zookeeper.test.properties @@ -0,0 +1,20 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# the directory where the snapshot is stored. +dataDir=/tmp/secor_dev/zookeeper +# the port at which the clients will connect +clientPort=2181 +# disable the per-ip limit on the number of connections since this is a non-production config +maxClientCnxns=0 diff --git a/src/main/java/com/pinterest/secor/common/FileRegistry.java b/src/main/java/com/pinterest/secor/common/FileRegistry.java new file mode 100644 index 000000000..4ded5a6de --- /dev/null +++ b/src/main/java/com/pinterest/secor/common/FileRegistry.java @@ -0,0 +1,232 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.common; + +import com.pinterest.secor.util.FileUtil; +import com.pinterest.secor.util.StatsUtil; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.SequenceFile; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.*; + +/** + * FileRegistry keeps track of local log files currently being appended to and the associated + * writers. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class FileRegistry { + private static final Logger LOG = LoggerFactory.getLogger(FileRegistry.class); + + private HashMap> mFiles; + private HashMap mWriters; + private HashMap mCreationTimes; + + public FileRegistry() { + mFiles = new HashMap>(); + mWriters = new HashMap(); + mCreationTimes = new HashMap(); + } + + /** + * Get all topic partitions. + * @return Collection of all registered topic partitions. + */ + public Collection getTopicPartitions() { + Set topicPartitions = mFiles.keySet(); + if (topicPartitions == null) { + return new HashSet(); + } + // Return a copy of the collection to prevent the caller from modifying internals. + return new HashSet(topicPartitions); + } + + /** + * Get paths in a given topic partition. + * @param topicPartition The topic partition to retrieve paths for. + * @return Collection of file paths in the given topic partition. + */ + public Collection getPaths(TopicPartition topicPartition) { + HashSet logFilePaths = mFiles.get(topicPartition); + if (logFilePaths == null) { + return new HashSet(); + } + return new HashSet(logFilePaths); + } + + /** + * Retrieve a writer for a given path or create a new one if it does not exist. + * @param path The path to retrieve writer for. + * @return Writer for a given path. + * @throws IOException + */ + public SequenceFile.Writer getOrCreateWriter(LogFilePath path) throws IOException { + SequenceFile.Writer writer = mWriters.get(path); + if (writer == null) { + // Just in case. + FileUtil.delete(path.getLogFilePath()); + FileUtil.delete(path.getLogFileCrcPath()); + TopicPartition topicPartition = new TopicPartition(path.getTopic(), + path.getKafkaPartition()); + HashSet files = mFiles.get(topicPartition); + if (files == null) { + files = new HashSet(); + mFiles.put(topicPartition, files); + } + if (!files.contains(path)) { + files.add(path); + } + Configuration config = new Configuration(); + Path fsPath = new Path(path.getLogFilePath()); + FileSystem fs = FileSystem.get(config); + writer = SequenceFile.createWriter(fs, config, fsPath, LongWritable.class, + BytesWritable.class); + mWriters.put(path, writer); + mCreationTimes.put(path, System.currentTimeMillis() / 1000L); + LOG.debug("created writer for path " + path.getLogFilePath()); + } + return writer; + } + + /** + * Delete a given path, the underlying file, and the corresponding writer. + * @param path The path to delete. + * @throws IOException + */ + public void deletePath(LogFilePath path) throws IOException { + TopicPartition topicPartition = new TopicPartition(path.getTopic(), + path.getKafkaPartition()); + HashSet paths = mFiles.get(topicPartition); + paths.remove(path); + if (paths.isEmpty()) { + mFiles.remove(topicPartition); + StatsUtil.clearLabel("secor.size." + topicPartition.getTopic() + "." + + topicPartition.getPartition()); + StatsUtil.clearLabel("secor.modification_age_sec." + topicPartition.getTopic() + "." + + topicPartition.getPartition()); + } + deleteWriter(path); + FileUtil.delete(path.getLogFilePath()); + FileUtil.delete(path.getLogFileCrcPath()); + } + + /** + * Delete all paths, files, and writers in a given topic partition. + * @param topicPartition The topic partition to remove. + * @throws IOException + */ + public void deleteTopicPartition(TopicPartition topicPartition) throws IOException { + HashSet paths = mFiles.get(topicPartition); + if (paths == null) { + return; + } + HashSet clonedPaths = (HashSet) paths.clone(); + for (LogFilePath path : clonedPaths) { + deletePath(path); + } + } + + /** + * Delete writer for a given topic partition. Underlying file is not removed. + * @param path The path to remove the writer for. + */ + public void deleteWriter(LogFilePath path) throws IOException { + SequenceFile.Writer writer = mWriters.get(path); + if (writer == null) { + LOG.warn("No writer found for path " + path.getLogFilePath()); + } else { + LOG.info("Deleting writer for path " + path.getLogFilePath()); + writer.close(); + mWriters.remove(path); + mCreationTimes.remove(path); + } + } + + /** + * Delete all writers in a given topic partition. Underlying files are not removed. + * @param topicPartition The topic partition to remove the writers for. + */ + public void deleteWriters(TopicPartition topicPartition) throws IOException { + HashSet paths = mFiles.get(topicPartition); + if (paths == null) { + LOG.warn("No paths found for topic " + topicPartition.getTopic() + " partition " + + topicPartition.getPartition()); + } else { + for (LogFilePath path : paths) { + deleteWriter(path); + } + } + } + + /** + * Get aggregated size of all files in a given topic partition. + * @param topicPartition The topic partition to get the size for. + * @return Aggregated size of files in the topic partition or 0 if the topic partition does + * not contain any files. + * @throws IOException + */ + public long getSize(TopicPartition topicPartition) throws IOException { + Collection paths = getPaths(topicPartition); + long result = 0; + for (LogFilePath path : paths) { + SequenceFile.Writer writer = mWriters.get(path); + if (writer != null) { + result += writer.getLength(); + } + } + StatsUtil.setLabel("secor.size." + topicPartition.getTopic() + "." + + topicPartition.getPartition(), Long.toString(result)); + return result; + } + + /** + * Get the creation age of the most recently created file in a given topic partition. + * @param topicPartition The topic partition to get the age of. + * @return Age of the most recently created file in the topic partition or -1 if the partition + * does not contain any files. + * @throws IOException + */ + public long getModificationAgeSec(TopicPartition topicPartition) throws IOException { + long now = System.currentTimeMillis() / 1000L; + long result = Long.MAX_VALUE; + Collection paths = getPaths(topicPartition); + for (LogFilePath path : paths) { + Long creationTime = mCreationTimes.get(path); + if (creationTime == null) { + LOG.warn("no creation time found for path " + path); + creationTime = now; + } + long age = now - creationTime; + if (age < result) { + result = age; + } + } + if (result == Long.MAX_VALUE) { + result = -1; + } + StatsUtil.setLabel("secor.modification_age_sec." + topicPartition.getTopic() + "." + + topicPartition.getPartition(), Long.toString(result)); + return result; + } +} diff --git a/src/main/java/com/pinterest/secor/common/KafkaClient.java b/src/main/java/com/pinterest/secor/common/KafkaClient.java new file mode 100644 index 000000000..3fe96a268 --- /dev/null +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -0,0 +1,187 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.common; + +import com.google.common.net.HostAndPort; +import com.pinterest.secor.message.Message; +import kafka.api.*; +import kafka.common.TopicAndPartition; +import kafka.javaapi.FetchResponse; +import kafka.javaapi.OffsetRequest; +import kafka.javaapi.OffsetResponse; +import kafka.javaapi.PartitionMetadata; +import kafka.javaapi.TopicMetadata; +import kafka.javaapi.TopicMetadataRequest; +import kafka.javaapi.TopicMetadataResponse; +import kafka.javaapi.consumer.SimpleConsumer; +import kafka.message.MessageAndOffset; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Kafka client encapsulates the logic interacting with Kafka brokers. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class KafkaClient { + private static final Logger LOG = LoggerFactory.getLogger(KafkaClient.class); + + private SecorConfig mConfig; + private ZookeeperConnector mZookeeperConnector; + + public KafkaClient(SecorConfig config) { + mConfig = config; + mZookeeperConnector = new ZookeeperConnector(mConfig); + } + + private HostAndPort findLeader(TopicPartition topicPartition) { + SimpleConsumer consumer = null; + try { + LOG.info("looking up lader for topic " + topicPartition.getTopic() + " partition " + + topicPartition.getPartition()); + consumer = new SimpleConsumer(mConfig.getKafkaSeedBrokerHost(), + mConfig.getKafkaSeedBrokerPort(), + 100000, 64 * 1024, "leaderLookup"); + List topics = new ArrayList(); + topics.add(topicPartition.getTopic()); + TopicMetadataRequest request = new TopicMetadataRequest(topics); + TopicMetadataResponse response = consumer.send(request); + + List metaData = response.topicsMetadata(); + for (TopicMetadata item : metaData) { + for (PartitionMetadata part : item.partitionsMetadata()) { + if (part.partitionId() == topicPartition.getPartition()) { + return HostAndPort.fromParts(part.leader().host(), part.leader().port()); + } + } + } + } finally { + if (consumer != null) { + consumer.close(); + } + } + return null; + } + + private static String getClientName(TopicPartition topicPartition) { + return "secorClient_" + topicPartition.getTopic() + "_" + topicPartition.getPartition(); + } + + private long findLastOffset(TopicPartition topicPartition, SimpleConsumer consumer) { + TopicAndPartition topicAndPartition = new TopicAndPartition(topicPartition.getTopic(), + topicPartition.getPartition()); + Map requestInfo = + new HashMap(); + requestInfo.put(topicAndPartition, new PartitionOffsetRequestInfo( + kafka.api.OffsetRequest.LatestTime(), 1)); + final String clientName = getClientName(topicPartition); + OffsetRequest request = new OffsetRequest(requestInfo, + kafka.api.OffsetRequest.CurrentVersion(), + clientName); + OffsetResponse response = consumer.getOffsetsBefore(request); + + if (response.hasError()) { + throw new RuntimeException("Error fetching offset data. Reason: " + + response.errorCode(topicPartition.getTopic(), topicPartition.getPartition())); + } + long[] offsets = response.offsets(topicPartition.getTopic(), + topicPartition.getPartition()); + return offsets[0] - 1; + } + + private Message getMessage(TopicPartition topicPartition, long offset, + SimpleConsumer consumer) { + LOG.info("fetching message topic " + topicPartition.getTopic() + " partition " + + topicPartition.getPartition() + " offset " + offset); + final int MAX_MESSAGE_SIZE_BYTES = 100000; + final String clientName = getClientName(topicPartition); + kafka.api.FetchRequest request = new FetchRequestBuilder().clientId(clientName) + .addFetch(topicPartition.getTopic(), topicPartition.getPartition(), offset, + MAX_MESSAGE_SIZE_BYTES) + .build(); + FetchResponse response = consumer.fetch(request); + if (response.hasError()) { + consumer.close(); + throw new RuntimeException("Error fetching offset data. Reason: " + + response.errorCode(topicPartition.getTopic(), topicPartition.getPartition())); + } + MessageAndOffset messageAndOffset = response.messageSet( + topicPartition.getTopic(), topicPartition.getPartition()).iterator().next(); + ByteBuffer payload = messageAndOffset.message().payload(); + byte[] payloadBytes = new byte[payload.limit()]; + payload.get(payloadBytes); + return new Message(topicPartition.getTopic(), topicPartition.getPartition(), + messageAndOffset.offset(), payloadBytes); + } + + private SimpleConsumer createConsumer(TopicPartition topicPartition) { + HostAndPort leader = findLeader(topicPartition); + LOG.info("leader for topic " + topicPartition.getTopic() + " partition " + + topicPartition.getPartition() + " is " + leader.toString()); + final String clientName = getClientName(topicPartition); + return new SimpleConsumer(leader.getHostText(), leader.getPort(), 100000, 64 * 1024, + clientName); + } + + public int getNumPartitions(String topic) { + SimpleConsumer consumer = null; + try { + consumer = new SimpleConsumer(mConfig.getKafkaSeedBrokerHost(), + mConfig.getKafkaSeedBrokerPort(), + 100000, 64 * 1024, "partitionLookup"); + List topics = new ArrayList(); + topics.add(topic); + TopicMetadataRequest request = new TopicMetadataRequest(topics); + TopicMetadataResponse response = consumer.send(request); + if (response.topicsMetadata().size() != 1) { + throw new RuntimeException("Expected one metadata for topic " + topic + " found " + + response.topicsMetadata().size()); + } + TopicMetadata topicMetadata = response.topicsMetadata().get(0); + return topicMetadata.partitionsMetadata().size(); + } finally { + if (consumer != null) { + consumer.close(); + } + } + } + + public Message getLastMessage(TopicPartition topicPartition) throws TException { + SimpleConsumer consumer = createConsumer(topicPartition); + long lastOffset = findLastOffset(topicPartition, consumer); + if (lastOffset < 1) { + return null; + } + return getMessage(topicPartition, lastOffset, consumer); + } + + public Message getCommittedMessage(TopicPartition topicPartition) throws Exception { + long committedOffset = mZookeeperConnector.getCommittedOffsetCount(topicPartition) - 1; + if (committedOffset < 0) { + return null; + } + SimpleConsumer consumer = createConsumer(topicPartition); + return getMessage(topicPartition, committedOffset, consumer); + } +} diff --git a/src/main/java/com/pinterest/secor/common/LogFilePath.java b/src/main/java/com/pinterest/secor/common/LogFilePath.java new file mode 100644 index 000000000..2815f6b6f --- /dev/null +++ b/src/main/java/com/pinterest/secor/common/LogFilePath.java @@ -0,0 +1,204 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.common; + +import com.pinterest.secor.message.ParsedMessage; +import org.apache.commons.lang.StringUtils; + +import java.util.ArrayList; +import java.util.Arrays; + +/** + * LogFilePath represents path of a log file. It contains convenience method for building and + * decomposing paths. + * + * Log file path has the following form: + * prefix/topic/partition1/.../partitionN/generation_kafkaParition_firstMessageOffset + * where: + * prefix is top-level directory for log files. It can be a local path or an s3 dir, + * topic is a kafka topic, + * partition1, ..., partitionN is the list of partition names extracted from message content. + * E.g., the partition may describe the message date such as dt=2014-01-01, + * generation is the consumer version. It allows up to perform rolling upgrades of + * non-compatible Secor releases, + * kafkaPartition is the kafka partition of the topic, + * firstMessageOffset is the offset of the first message in a batch of files committed + * atomically. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class LogFilePath { + private String mPrefix; + private String mTopic; + private String[] mPartitions; + private int mGeneration; + private int mKafkaPartition; + private long mOffset; + + public LogFilePath(String prefix, int generation, long lastCommittedOffset, + ParsedMessage message) { + mPrefix = prefix; + mTopic = message.getTopic(); + mPartitions = message.getPartitions(); + mGeneration = generation; + mKafkaPartition = message.getKafkaPartition(); + mOffset = lastCommittedOffset; + } + + public LogFilePath(String prefix, String topic, String[] partitions, int generation, + int kafkaPartition, long offset) { + mPrefix = prefix; + mTopic = topic; + mPartitions = partitions; + mGeneration = generation; + mKafkaPartition = kafkaPartition; + mOffset = offset; + } + + private static String[] subArray(String[] array, int startIndex, int endIndex) { + String[] result = new String[endIndex - startIndex + 1]; + for (int i = startIndex; i <= endIndex; ++i) { + result[i - startIndex] = array[i]; + } + return result; + } + + public LogFilePath(String prefix, String path) { + assert path.startsWith(prefix): path + ".startsWith(" + prefix + ")"; + + mPrefix = prefix; + + int prefixLength = prefix.length(); + if (!prefix.endsWith("/")) { + prefixLength++; + } + String suffix = path.substring(prefixLength); + String[] pathElements = suffix.split("/"); + // Suffix should contain a topic, at least one partition, and the basename. + assert pathElements.length >= 3: Arrays.toString(pathElements) + ".length >= 3"; + + mTopic = pathElements[0]; + mPartitions = subArray(pathElements, 1, pathElements.length - 2); + + // Parse basename. + String basename = pathElements[pathElements.length - 1]; + // Remove extension. + basename = basename.split("\\.")[0]; + String[] basenameElements = basename.split("_"); + assert basenameElements.length == 3: Integer.toString(basenameElements.length) + " == 3"; + mGeneration = Integer.parseInt(basenameElements[0]); + mKafkaPartition = Integer.parseInt(basenameElements[1]); + mOffset = Long.parseLong(basenameElements[2]); + } + + public String getLogFileParentDir() { + ArrayList elements = new ArrayList(); + elements.add(mPrefix); + elements.add(mTopic); + return StringUtils.join(elements, "/"); + } + + public String getLogFileDir() { + ArrayList elements = new ArrayList(); + elements.add(getLogFileParentDir()); + for (String partition : mPartitions) { + elements.add(partition); + } + return StringUtils.join(elements, "/"); + } + + private String getLogFileBasename() { + ArrayList basenameElements = new ArrayList(); + basenameElements.add(Integer.toString(mGeneration)); + basenameElements.add(Integer.toString(mKafkaPartition)); + basenameElements.add(String.format("%020d", mOffset)); + return StringUtils.join(basenameElements, "_"); + } + + public String getLogFilePath() { + String basename = getLogFileBasename(); + + ArrayList pathElements = new ArrayList(); + pathElements.add(getLogFileDir()); + pathElements.add(basename); + + return StringUtils.join(pathElements, "/"); + } + + public String getLogFileCrcPath() { + String basename = "." + getLogFileBasename() + ".crc"; + + ArrayList pathElements = new ArrayList(); + pathElements.add(getLogFileDir()); + pathElements.add(basename); + + return StringUtils.join(pathElements, "/"); + } + + public String getTopic() { + return mTopic; + } + + public String[] getPartitions() { + return mPartitions; + } + + public int getGeneration() { + return mGeneration; + } + + public int getKafkaPartition() { + return mKafkaPartition; + } + + public long getOffset() { + return mOffset; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + LogFilePath that = (LogFilePath) o; + + if (mGeneration != that.mGeneration) return false; + if (mKafkaPartition != that.mKafkaPartition) return false; + if (mOffset != that.mOffset) return false; + if (!Arrays.equals(mPartitions, that.mPartitions)) return false; + if (mPrefix != null ? !mPrefix.equals(that.mPrefix) : that.mPrefix != null) return false; + if (mTopic != null ? !mTopic.equals(that.mTopic) : that.mTopic != null) return false; + + return true; + } + + @Override + public int hashCode() { + int result = mPrefix != null ? mPrefix.hashCode() : 0; + result = 31 * result + (mTopic != null ? mTopic.hashCode() : 0); + result = 31 * result + (mPartitions != null ? Arrays.hashCode(mPartitions) : 0); + result = 31 * result + mGeneration; + result = 31 * result + mKafkaPartition; + result = 31 * result + (int) (mOffset ^ (mOffset >>> 32)); + return result; + } + + @Override + public String toString() { + return getLogFilePath(); + } +} diff --git a/src/main/java/com/pinterest/secor/common/OffsetTracker.java b/src/main/java/com/pinterest/secor/common/OffsetTracker.java new file mode 100644 index 000000000..f67a4cc4d --- /dev/null +++ b/src/main/java/com/pinterest/secor/common/OffsetTracker.java @@ -0,0 +1,91 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.common; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; + +/** + * Offset tracker stores offset related metadata. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class OffsetTracker { + private static final Logger LOG = LoggerFactory.getLogger(OffsetTracker.class); + + private HashMap mLastSeenOffset; + private HashMap mFirstSeendOffset; + private HashMap mCommittedOffsetCount; + + public OffsetTracker() { + mLastSeenOffset = new HashMap(); + mCommittedOffsetCount = new HashMap(); + mFirstSeendOffset = new HashMap(); + } + + public long getLastSeenOffset(TopicPartition topicPartition) { + Long offset = mLastSeenOffset.get(topicPartition); + if (offset == null) { + return -2; + } + return offset.longValue(); + } + + public long setLastSeenOffset(TopicPartition topicPartition, long offset) { + long lastSeenOffset = getLastSeenOffset(topicPartition); + mLastSeenOffset.put(topicPartition, offset); + if (lastSeenOffset + 1 != offset) { + LOG.warn("offset for topic " + topicPartition.getTopic() + " partition " + + topicPartition.getPartition() + " changed from " + lastSeenOffset + " to " + + offset); + } + if (mFirstSeendOffset.get(topicPartition) == null) { + mFirstSeendOffset.put(topicPartition, offset); + } + return lastSeenOffset; + } + + public long getTrueCommittedOffsetCount(TopicPartition topicPartition) { + Long committedOffsetCount = mCommittedOffsetCount.get(topicPartition); + if (committedOffsetCount == null) { + return -1L; + } + return committedOffsetCount; + } + + public long getAdjustedCommittedOffsetCount(TopicPartition topicPartition) { + long trueCommittedOffsetCount = getTrueCommittedOffsetCount(topicPartition); + if (trueCommittedOffsetCount == -1L) { + Long firstSeenOffset = mFirstSeendOffset.get(topicPartition); + if (firstSeenOffset != null) { + return firstSeenOffset; + } + } + return trueCommittedOffsetCount; + } + + public long setCommittedOffsetCount(TopicPartition topicPartition, long count) { + long trueCommittedOffsetCount = getTrueCommittedOffsetCount(topicPartition); + // Committed offsets should never go back. + assert trueCommittedOffsetCount <= count: Long.toString(trueCommittedOffsetCount) + + " <= " + count; + mCommittedOffsetCount.put(topicPartition, count); + return trueCommittedOffsetCount; + } +} diff --git a/src/main/java/com/pinterest/secor/common/OstrichAdminService.java b/src/main/java/com/pinterest/secor/common/OstrichAdminService.java new file mode 100644 index 000000000..cc2adebbb --- /dev/null +++ b/src/main/java/com/pinterest/secor/common/OstrichAdminService.java @@ -0,0 +1,71 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.common; + +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +import com.pinterest.secor.util.StatsUtil; +import com.twitter.ostrich.admin.*; +import com.twitter.util.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Option; +import scala.collection.Map$; +import scala.collection.immutable.List; +import scala.collection.immutable.List$; +import scala.util.matching.Regex; + +/** + * OstrichAdminService initializes export of metrics to Ostrich. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class OstrichAdminService { + private static final Logger LOG = LoggerFactory.getLogger(OstrichAdminService.class); + private final int mPort; + + public OstrichAdminService(int port) { + this.mPort = port; + } + + public void start() { + Duration[] defaultLatchIntervals = {Duration.apply(1, TimeUnit.MINUTES)}; + @SuppressWarnings("deprecation") + AdminServiceFactory adminServiceFactory = new AdminServiceFactory( + this.mPort, + 20, + List$.MODULE$.empty(), + Option.empty(), + List$.MODULE$.empty(), + Map$.MODULE$.empty(), + List.fromArray(defaultLatchIntervals) + ); + RuntimeEnvironment runtimeEnvironment = new RuntimeEnvironment(this); + adminServiceFactory.apply(runtimeEnvironment); + try { + Properties properties = new Properties(); + properties.load(this.getClass().getResource("build.properties").openStream()); + String buildRevision = properties.getProperty("build_revision", "unknown"); + LOG.info("build.properties build_revision: {}", + properties.getProperty("build_revision", "unknown")); + StatsUtil.setLabel("secor.build_revision", buildRevision); + } catch (Throwable t) { + LOG.error("Failed to load properties from build.properties", t); + } + } +} diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java new file mode 100644 index 000000000..49060e97f --- /dev/null +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -0,0 +1,163 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.common; + +import org.apache.commons.configuration.ConfigurationException; +import org.apache.commons.configuration.PropertiesConfiguration; +import org.apache.commons.lang.StringUtils; + +/** + * One-stop shop for Secor configuration options. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class SecorConfig { + private final PropertiesConfiguration mProperties; + + public static SecorConfig load() throws ConfigurationException { + // Load the default configuration file first + String configProperty = System.getProperty("config"); + PropertiesConfiguration properties = new PropertiesConfiguration(configProperty); + + return new SecorConfig(properties); + } + + private SecorConfig(PropertiesConfiguration properties) { + mProperties = properties; + } + + public String getKafkaSeedBrokerHost() { + return getString("kafka.seed.broker.host"); + } + + public int getKafkaSeedBrokerPort() { + return getInt("kafka.seed.broker.port"); + } + + public String getZookeeperQuorum() { + return StringUtils.join(getStringArray("zookeeper.quorum"), ','); + } + + public int getConsumerTimeoutMs() { + return getInt("kafka.consumer.timeout.ms"); + } + + public int getGeneration() { + return getInt("secor.generation"); + } + + public int getConsumerThreads() { + return getInt("secor.consumer.threads"); + } + + public long getMaxFileSizeBytes() { + return getLong("secor.max.file.size.bytes"); + } + + public long getMaxFileAgeSeconds() { + return getLong("secor.max.file.age.seconds"); + } + + public long getOffsetsPerPartition() { + return getLong("secor.offsets.per.partition"); + } + + public int getMessagesPerSecond() { + return getInt("secor.messages.per.second"); + } + + public String getS3Bucket() { + return getString("secor.s3.bucket"); + } + + public String getS3Path() { + return getString("secor.s3.path"); + } + + public String getLocalPath() { + return getString("secor.local.path"); + } + + public String getKafkaTopicFilter() { + return getString("secor.kafka.topic_filter"); + } + + public String getKafkaGroup() { + return getString("secor.kafka.group"); + } + + public int getZookeeperSessionTimeoutMs() { + return getInt("zookeeper.session.timeout.ms"); + } + + public int getZookeeperSyncTimeMs() { + return getInt("zookeeper.sync.time.ms"); + } + + public String getMessageParserClass() { + return getString("secor.message.parser.class"); + } + + public int getTopicPartitionForgetSeconds() { + return getInt("secor.topic_partition.forget.seconds"); + } + + public int getOstrichPort() { + return getInt("ostrich.port"); + } + + public String getAwsAccessKey() { + return getString("aws.access.key"); + } + + public String getAwsSecretKey() { + return getString("aws.secret.key"); + } + + public String getQuboleApiToken() { + return getString("qubole.api.token"); + } + + public String getTsdbHostport() { + return getString("tsdb.hostport"); + } + + private void checkProperty(String name) { + if (!mProperties.containsKey(name)) { + throw new RuntimeException("Failed to find required configuration option '" + + name + "'."); + } + } + + private String getString(String name) { + checkProperty(name); + return mProperties.getString(name); + } + + private int getInt(String name) { + checkProperty(name); + return mProperties.getInt(name); + } + + private long getLong(String name) { + return mProperties.getLong(name); + } + + private String[] getStringArray(String name) { + return mProperties.getStringArray(name); + } +} diff --git a/src/main/java/com/pinterest/secor/common/TopicPartition.java b/src/main/java/com/pinterest/secor/common/TopicPartition.java new file mode 100644 index 000000000..03e660326 --- /dev/null +++ b/src/main/java/com/pinterest/secor/common/TopicPartition.java @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.common; + +/** + * Topic partition describes a kafka message topic-partition pair. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class TopicPartition { + private String mTopic; + private int mPartition; + + public TopicPartition(String topic, int partition) { + mTopic = topic; + mPartition = partition; + } + + public String getTopic() { + return mTopic; + } + + public int getPartition() { + return mPartition; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + TopicPartition that = (TopicPartition) o; + + if (mPartition != that.mPartition) return false; + if (mTopic != null ? !mTopic.equals(that.mTopic) : that.mTopic != null) return false; + + return true; + } + + @Override + public int hashCode() { + int result = mTopic != null ? mTopic.hashCode() : 0; + result = 31 * result + mPartition; + return result; + } + + @Override + public String toString() { + return "TopicPartition{" + + "mTopic='" + mTopic + '\'' + + ", mPartition=" + mPartition + + '}'; + } +} diff --git a/src/main/java/com/pinterest/secor/common/ZookeeperConnector.java b/src/main/java/com/pinterest/secor/common/ZookeeperConnector.java new file mode 100644 index 000000000..20c3e56d0 --- /dev/null +++ b/src/main/java/com/pinterest/secor/common/ZookeeperConnector.java @@ -0,0 +1,182 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.common; + +import com.twitter.common.quantity.Amount; +import com.twitter.common.quantity.Time; +import com.twitter.common.zookeeper.DistributedLock; +import com.twitter.common.zookeeper.DistributedLockImpl; +import com.twitter.common.zookeeper.ZooKeeperClient; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.InetSocketAddress; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; + +/** + * ZookeeperConnector implements interactions with Zookeeper. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class ZookeeperConnector { + private static final Logger LOG = LoggerFactory.getLogger(ZookeeperConnector.class); + + private SecorConfig mConfig; + private ZooKeeperClient mZookeeperClient; + private HashMap mLocks; + + public ZookeeperConnector(SecorConfig config) { + mConfig = config; + mZookeeperClient = new ZooKeeperClient(Amount.of(1, Time.DAYS), getZookeeperAddresses()); + mLocks = new HashMap(); + } + + private Iterable getZookeeperAddresses() { + String zookeeperQuorum = mConfig.getZookeeperQuorum(); + String[] hostports = zookeeperQuorum.split(","); + LinkedList result = new LinkedList(); + for (String hostport : hostports) { + String[] elements = hostport.split(":"); + assert elements.length == 2: Integer.toString(elements.length) + " == 2"; + String host = elements[0]; + int port = Integer.parseInt(elements[1]); + result.add(new InetSocketAddress(host, port)); + } + return result; + } + + public void lock(String lockPath) { + assert mLocks.get(lockPath) == null: "mLocks.get(" + lockPath + ") == null"; + DistributedLock distributedLock = new DistributedLockImpl(mZookeeperClient, lockPath); + mLocks.put(lockPath, distributedLock); + distributedLock.lock(); + } + + public void unlock(String lockPath) { + DistributedLock distributedLock = mLocks.get(lockPath); + assert distributedLock != null: "mLocks.get(" + lockPath + ") != null"; + distributedLock.unlock(); + mLocks.remove(lockPath); + } + + private String getCommittedOffsetGroupPath() { + return "/consumers/" + mConfig.getKafkaGroup() + "/offsets"; + } + + private String getCommittedOffsetTopicPath(String topic) { + return getCommittedOffsetGroupPath() + "/" + topic; + } + + private String getCommittedOffsetPartitionPath(TopicPartition topicPartition) { + return getCommittedOffsetTopicPath(topicPartition.getTopic()) + "/" + + topicPartition.getPartition(); + } + + public long getCommittedOffsetCount(TopicPartition topicPartition) throws Exception { + ZooKeeper zookeeper = mZookeeperClient.get(); + String offsetPath = getCommittedOffsetPartitionPath(topicPartition); + try { + byte[] data = zookeeper.getData(offsetPath, false, null); + return Long.parseLong(new String(data)); + } catch (KeeperException.NoNodeException exception) { + LOG.warn("path " + offsetPath + " does not exist in zookeeper"); + return -1; + } + } + + public List getCommittedOffsetPartitions(String topic) throws Exception { + ZooKeeper zookeeper = mZookeeperClient.get(); + String topicPath = getCommittedOffsetTopicPath(topic); + List partitions = zookeeper.getChildren(topicPath, false); + LinkedList result = new LinkedList(); + for (String partitionPath : partitions) { + String[] elements = partitionPath.split("/"); + String partition = elements[elements.length - 1]; + result.add(Integer.valueOf(partition)); + } + return result; + } + + public List getCommittedOffsetTopics() throws Exception { + ZooKeeper zookeeper = mZookeeperClient.get(); + String offsetPath = getCommittedOffsetGroupPath(); + List topics = zookeeper.getChildren(offsetPath, false); + LinkedList result = new LinkedList(); + for (String topicPath : topics) { + String[] elements = topicPath.split("/"); + String topic = elements[elements.length - 1]; + result.add(topic); + } + return result; + } + + private void createMissingParents(String path) throws Exception { + ZooKeeper zookeeper = mZookeeperClient.get(); + assert path.charAt(0) == '/': path + ".charAt(0) == '/'"; + String[] elements = path.split("/"); + String prefix = ""; + for (int i = 1; i < elements.length - 1; ++i) { + prefix += "/" + elements[i]; + try { + zookeeper.create(prefix, null, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + LOG.info("created path " + prefix); + } catch (KeeperException.NodeExistsException exception) { + } + } + } + + public void setCommittedOffsetCount(TopicPartition topicPartition, long count) + throws Exception { + ZooKeeper zookeeper = mZookeeperClient.get(); + String offsetPath = getCommittedOffsetPartitionPath(topicPartition); + LOG.info("creating missing parents for zookeeper path " + offsetPath); + createMissingParents(offsetPath); + byte[] data = Long.toString(count).getBytes(); + try { + LOG.info("setting zookeeper path " + offsetPath + " value " + count); + // -1 matches any version + zookeeper.setData(offsetPath, data, -1); + } catch (KeeperException.NoNodeException exception) { + zookeeper.create(offsetPath, data, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + } + } + + public void deleteCommittedOffsetTopicCount(String topic) throws Exception { + ZooKeeper zookeeper = mZookeeperClient.get(); + List partitions = getCommittedOffsetPartitions(topic); + for (Integer partition : partitions) { + TopicPartition topicPartition = new TopicPartition(topic, partition); + String offsetPath = getCommittedOffsetPartitionPath(topicPartition); + LOG.info("deleting path " + offsetPath); + zookeeper.delete(offsetPath, -1); + } + } + + public void deleteCommittedOffsetPartitionCount(TopicPartition topicPartition) + throws Exception { + String offsetPath = getCommittedOffsetPartitionPath(topicPartition); + ZooKeeper zookeeper = mZookeeperClient.get(); + LOG.info("deleting path " + offsetPath); + zookeeper.delete(offsetPath, -1); + } +} diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java new file mode 100644 index 000000000..be2b9ee3a --- /dev/null +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -0,0 +1,117 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.consumer; + +import com.pinterest.secor.common.FileRegistry; +import com.pinterest.secor.common.OffsetTracker; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; +import com.pinterest.secor.message.ParsedMessage; +import com.pinterest.secor.parser.MessageParser; +import com.pinterest.secor.uploader.Uploader; +import com.pinterest.secor.reader.MessageReader; +import com.pinterest.secor.util.ReflectionUtil; +import com.pinterest.secor.writer.MessageWriter; +import kafka.consumer.ConsumerTimeoutException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.Thread; + +/** + * Consumer is a top-level component coordinating reading, writing, and uploading Kafka log + * messages. It is implemented as a thread with the intent of running multiple consumer + * concurrently. + * + * Note that consumer is not fixed with a specific topic partition. Kafka rebalancing mechanism + * allocates topic partitions to consumers dynamically to accommodate consumer population changes. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class Consumer extends Thread { + private static final Logger LOG = LoggerFactory.getLogger(Consumer.class); + + private SecorConfig mConfig; + + private MessageReader mMessageReader; + private MessageWriter mMessageWriter; + private MessageParser mMessageParser; + private Uploader mUploader; + + public Consumer(SecorConfig config) { + mConfig = config; + } + + private void init() throws Exception { + OffsetTracker offsetTracker = new OffsetTracker(); + mMessageReader = new MessageReader(mConfig, offsetTracker); + FileRegistry fileRegistry = new FileRegistry(); + mMessageWriter = new MessageWriter(mConfig, offsetTracker, fileRegistry); + mMessageParser = (MessageParser) ReflectionUtil.createMessageParser( + mConfig.getMessageParserClass(), mConfig); + mUploader = new Uploader(mConfig, offsetTracker, fileRegistry); + } + + @Override + public void run() { + try { + // init() cannot be called in the constructor since it contains logic dependent on the + // thread id. + init(); + } catch (Exception e) { + throw new RuntimeException("Failed to initialize the consumer", e); + } + while (true) { + Message rawMessage = null; + try { + boolean hasNext = mMessageReader.hasNext(); + if (!hasNext) { + return; + } + rawMessage = mMessageReader.read(); + } catch (ConsumerTimeoutException e) { + // We wait for a new message with a timeout to periodically apply the upload policy + // even if no messages are delivered. + } + if (rawMessage != null) { + ParsedMessage parsedMessage; + try { + parsedMessage = mMessageParser.parse(rawMessage); + } catch (Exception e) { + // TODO(pawel): we should tolerate a number of incorrectly formatted messages + // before crashing. + throw new RuntimeException("Failed to parse message " + rawMessage, e); + } + if (parsedMessage != null) { + try { + mMessageWriter.write(parsedMessage); + } catch (IOException e) { + throw new RuntimeException("Failed to write message " + parsedMessage, e); + } + } + } + // TODO(pawel): it may make sense to invoke the uploader less frequently than after + // each message. + try { + mUploader.applyPolicy(); + } catch (Exception e) { + throw new RuntimeException("Failed to apply upload policy", e); + } + } + } +} diff --git a/src/main/java/com/pinterest/secor/main/ConsumerMain.java b/src/main/java/com/pinterest/secor/main/ConsumerMain.java new file mode 100644 index 000000000..dbac8ae9c --- /dev/null +++ b/src/main/java/com/pinterest/secor/main/ConsumerMain.java @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.main; + +import com.pinterest.secor.common.OstrichAdminService; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.consumer.Consumer; +import com.pinterest.secor.util.FileUtil; +import com.pinterest.secor.util.RateLimitUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.LinkedList; + +/** + * Secor consumer. See + * https://docs.google.com/a/pinterest.com/document/d/1RHeH79O0e1WzsxumE24MIYqJFnRoRzQ3c74Wq3Q4R40/edit + * for detailed design. + * + * Run: + * $ cd optimus/secor + * $ mvn package + * $ cd target + * $ java -ea -Dlog4j.configuration=log4j.dev.properties -Dconfig=secor.dev.backup.properties \ + * -cp "secor-0.1-SNAPSHOT.jar:lib/*" com.pinterest.secor.main.ConsumerMain + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class ConsumerMain { + private static final Logger LOG = LoggerFactory.getLogger(ConsumerMain.class); + + public static void main(String[] args) { + if (args.length != 0) { + System.err.println("Usage: java -Dconfig= " + + "-Dlog4j.configuration= ConsumerMain"); + return; + } + try { + SecorConfig config = SecorConfig.load(); + OstrichAdminService ostrichService = new OstrichAdminService(config.getOstrichPort()); + ostrichService.start(); + FileUtil.configure(config); + RateLimitUtil.configure(config); + Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler() { + public void uncaughtException(Thread thread, Throwable exception) { + LOG.error("Thread " + thread + " failed", exception); + System.exit(1); + } + }; + LOG.info("starting " + config.getConsumerThreads() + " consumer threads"); + LinkedList consumers = new LinkedList(); + for (int i = 0; i < config.getConsumerThreads(); ++i) { + Consumer consumer = new Consumer(config); + consumer.setUncaughtExceptionHandler(handler); + consumers.add(consumer); + consumer.start(); + } + for (Consumer consumer : consumers) { + consumer.join(); + } + } catch (Throwable t) { + LOG.error("Consumer failed", t); + System.exit(1); + } + } +} diff --git a/src/main/java/com/pinterest/secor/main/LogFilePrinterMain.java b/src/main/java/com/pinterest/secor/main/LogFilePrinterMain.java new file mode 100644 index 000000000..deb751a97 --- /dev/null +++ b/src/main/java/com/pinterest/secor/main/LogFilePrinterMain.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.main; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.tools.LogFilePrinter; +import com.pinterest.secor.util.FileUtil; +import org.apache.commons.cli.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Log file printer main. + * + * Run: + * $ cd optimus/secor + * $ mvn package + * $ cd target + * $ java -ea -Dlog4j.configuration=log4j.dev.properties -Dconfig=secor.dev.backup.properties \ + * -cp "secor-0.1-SNAPSHOT.jar:lib/*" com.pinterest.secor.main.LogFilePrinterMain -f \ + * s3n://bucket/path + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class LogFilePrinterMain { + private static final Logger LOG = LoggerFactory.getLogger(LogFilePrinterMain.class); + + private static CommandLine parseArgs(String[] args) throws ParseException { + Options options = new Options(); + options.addOption(OptionBuilder.withLongOpt("file") + .withDescription("sequence file to read") + .hasArg() + .withArgName("") + .withType(String.class) + .create("f")); + options.addOption("o", "print_offsets_only", false, "whether to print only offsets " + + "ignoring the message payload"); + + CommandLineParser parser = new GnuParser(); + return parser.parse(options, args); + } + + public static void main(String[] args) { + try { + CommandLine commandLine = parseArgs(args); + SecorConfig config = SecorConfig.load(); + FileUtil.configure(config); + LogFilePrinter printer = new LogFilePrinter(config, + commandLine.hasOption("print_offsets_only")); + printer.printFile(commandLine.getOptionValue("file")); + } catch (Throwable t) { + LOG.error("Log file printer failed", t); + System.exit(1); + } + } +} diff --git a/src/main/java/com/pinterest/secor/main/LogFileVerifierMain.java b/src/main/java/com/pinterest/secor/main/LogFileVerifierMain.java new file mode 100644 index 000000000..ec1d7ec40 --- /dev/null +++ b/src/main/java/com/pinterest/secor/main/LogFileVerifierMain.java @@ -0,0 +1,103 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.main; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.tools.LogFileVerifier; +import com.pinterest.secor.util.FileUtil; +import org.apache.commons.cli.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Log file verifier main. + * + * Run: + * $ cd optimus/secor + * $ mvn package + * $ cd target + * $ java -ea -Dlog4j.configuration=log4j.dev.properties -Dconfig=secor.dev.backup.properties \ + * -cp "secor-0.1-SNAPSHOT.jar:lib/*" com.pinterest.secor.main.LogFileVerifierMain -t \ + * topic -q + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class LogFileVerifierMain { + private static final Logger LOG = LoggerFactory.getLogger(LogFileVerifierMain.class); + + private static CommandLine parseArgs(String[] args) throws ParseException { + Options options = new Options(); + options.addOption(OptionBuilder.withLongOpt("topic") + .withDescription("kafka topic name") + .hasArg() + .withArgName("") + .withType(String.class) + .create("t")); + options.addOption(OptionBuilder.withLongOpt("start_offset") + .withDescription("offset identifying the first set of files to check") + .withArgName("") + .withType(Long.class) + .create("s")); + options.addOption(OptionBuilder.withLongOpt("end_offset") + .withDescription("offset identifying the last set of files to check") + .withArgName("") + .withType(Long.class) + .create("e")); + options.addOption(OptionBuilder.withLongOpt("messages") + .withDescription("expected number of messages") + .hasArg() + .withArgName("") + .withType(Number.class) + .create("m")); + options.addOption("q", "sequence_offsets", false, "whether to verify that offsets " + + "increase sequentially. Requires loading all offsets in a snapshot " + + "to memory so use cautiously"); + + CommandLineParser parser = new GnuParser(); + return parser.parse(options, args); + } + + public static void main(String[] args) { + try { + CommandLine commandLine = parseArgs(args); + SecorConfig config = SecorConfig.load(); + FileUtil.configure(config); + LogFileVerifier verifier = new LogFileVerifier(config, + commandLine.getOptionValue("topic")); + long startOffset = -2; + long endOffset = Long.MAX_VALUE; + if (commandLine.hasOption("start_offset")) { + startOffset = Long.parseLong(commandLine.getOptionValue("start_offset")); + if (commandLine.hasOption("end_offset")) { + endOffset = Long.parseLong(commandLine.getOptionValue("end_offset")); + } + } + int numMessages = -1; + if (commandLine.hasOption("messages")) { + numMessages = ((Number) commandLine.getParsedOptionValue("messages")).intValue(); + } + verifier.verifyCounts(startOffset, endOffset, numMessages); + if (commandLine.hasOption("sequence_offsets")) { + verifier.verifySequences(startOffset, endOffset); + } + System.out.println("verification succeeded"); + } catch (Throwable t) { + LOG.error("Log file verifier failed", t); + System.exit(1); + } + } +} diff --git a/src/main/java/com/pinterest/secor/main/PartitionFinalizerMain.java b/src/main/java/com/pinterest/secor/main/PartitionFinalizerMain.java new file mode 100644 index 000000000..77b45b58a --- /dev/null +++ b/src/main/java/com/pinterest/secor/main/PartitionFinalizerMain.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.main; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.parser.PartitionFinalizer; +import com.pinterest.secor.util.FileUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Partition finalizer main. + * + * Run: + * $ cd optimus/secor + * $ mvn package + * $ cd target + * $ java -ea -Dlog4j.configuration=log4j.dev.properties -Dconfig=secor.dev.backup.properties \ + * -cp "secor-0.1-SNAPSHOT.jar:lib/*" com.pinterest.secor.main.PartitionFinalizerMain + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class PartitionFinalizerMain { + private static final Logger LOG = LoggerFactory.getLogger(LogFilePrinterMain.class); + + public static void main(String[] args) { + try { + SecorConfig config = SecorConfig.load(); + FileUtil.configure(config); + PartitionFinalizer partitionFinalizer = new PartitionFinalizer(config); + partitionFinalizer.finalizePartitions(); + } catch (Throwable t) { + LOG.error("Partition finalizer failed", t); + System.exit(1); + } + } +} diff --git a/src/main/java/com/pinterest/secor/main/ProgressMonitorMain.java b/src/main/java/com/pinterest/secor/main/ProgressMonitorMain.java new file mode 100644 index 000000000..425a1ef3f --- /dev/null +++ b/src/main/java/com/pinterest/secor/main/ProgressMonitorMain.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.main; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.tools.ProgressMonitor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Progress monitor main. + * + * Run: + * $ cd optimus/secor + * $ mvn package + * $ cd target + * $ java -ea -Dlog4j.configuration=log4j.dev.properties -Dconfig=secor.dev.backup.properties \ + * -cp "secor-0.1-SNAPSHOT.jar:lib/*" com.pinterest.secor.main.ProgressMonitorMain + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class ProgressMonitorMain { + private static final Logger LOG = LoggerFactory.getLogger(ProgressMonitorMain.class); + + public static void main(String[] args) { + try { + SecorConfig config = SecorConfig.load(); + ProgressMonitor progressMonitor = new ProgressMonitor(config); + progressMonitor.exportStats(); + } catch (Throwable t) { + LOG.error("Progress monitor failed", t); + System.exit(1); + } + } +} diff --git a/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java b/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java new file mode 100644 index 000000000..192dac091 --- /dev/null +++ b/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.main; + +import com.pinterest.secor.tools.TestLogMessageProducer; +import org.apache.commons.cli.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test log message producer main. + * + * Run: + * $ cd optimus/secor + * $ mvn package + * $ cd target + * $ java -ea -Dlog4j.configuration=log4j.dev.properties -Dconfig=secor.dev.backup.properties \ + * -cp "secor-0.1-SNAPSHOT.jar:lib/*" com.pinterest.secor.main.TestLogMessageProducerMain \ + * -t topic -m num_messages -p num_producer_threads + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class TestLogMessageProducerMain { + private static final Logger LOG = LoggerFactory.getLogger(TestLogMessageProducerMain.class); + + private static CommandLine parseArgs(String[] args) throws ParseException { + Options options = new Options(); + options.addOption(OptionBuilder.withLongOpt("topic") + .withDescription("topic to post to") + .hasArg() + .withArgName("") + .withType(String.class) + .create("t")); + options.addOption(OptionBuilder.withLongOpt("messages") + .withDescription("number of messages per producer to post") + .hasArg() + .withArgName("") + .withType(Number.class) + .create("m")); + options.addOption(OptionBuilder.withLongOpt("producers") + .withDescription("number of producer threads") + .hasArg() + .withArgName("") + .withType(Number.class) + .create("p")); + + CommandLineParser parser = new GnuParser(); + return parser.parse(options, args); + } + + public static void main(String[] args) { + try { + CommandLine commandLine = parseArgs(args); + String topic = commandLine.getOptionValue("topic"); + int messages = ((Number) commandLine.getParsedOptionValue("messages")).intValue(); + int producers = ((Number) commandLine.getParsedOptionValue("producers")).intValue(); + for (int i = 0; i < producers; ++i) { + TestLogMessageProducer producer = new TestLogMessageProducer(topic, messages); + producer.start(); + } + } catch (Throwable t) { + LOG.error("Log message producer failed", t); + System.exit(1); + } + } +} diff --git a/src/main/java/com/pinterest/secor/main/ZookeeperClientMain.java b/src/main/java/com/pinterest/secor/main/ZookeeperClientMain.java new file mode 100644 index 000000000..946b0fdd1 --- /dev/null +++ b/src/main/java/com/pinterest/secor/main/ZookeeperClientMain.java @@ -0,0 +1,91 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.main; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.common.TopicPartition; +import com.pinterest.secor.common.ZookeeperConnector; +import org.apache.commons.cli.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Zookeeper client main. + * + * Run: + * $ cd optimus/secor + * $ mvn package + * $ cd target + * $ java -ea -Dlog4j.configuration=log4j.dev.properties -Dconfig=secor.dev.backup.properties \ + * -cp "secor-0.1-SNAPSHOT.jar:lib/*" com.pinterest.secor.main.ZookeeperClientMain -c \ + * delete_committed_offsets -t test -p 0 + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class ZookeeperClientMain { + private static final Logger LOG = LoggerFactory.getLogger(LogFilePrinterMain.class); + + private static CommandLine parseArgs(String[] args) throws ParseException { + Options options = new Options(); + options.addOption(OptionBuilder.withLongOpt("command") + .withDescription("command name. One of \"delete_committed_offsets\"") + .hasArg() + .withArgName("") + .withType(String.class) + .create("c")); + options.addOption(OptionBuilder.withLongOpt("topic") + .withDescription("topic whose offset should be read") + .hasArg() + .withArgName("") + .withType(String.class) + .create("t")); + options.addOption(OptionBuilder.withLongOpt("partition") + .withDescription("kafka partition whose offset should be read") + .hasArg() + .withArgName("") + .withType(Number.class) + .create("p")); + + CommandLineParser parser = new GnuParser(); + return parser.parse(options, args); + } + + public static void main(String[] args) { + try { + CommandLine commandLine = parseArgs(args); + String command = commandLine.getOptionValue("command"); + if (!command.equals("delete_committed_offsets")) { + throw new IllegalArgumentException( + "command has to be one of \"delete_committed_offsets\""); + } + SecorConfig config = SecorConfig.load(); + ZookeeperConnector zookeeperConnector = new ZookeeperConnector(config); + String topic = commandLine.getOptionValue("topic"); + if (commandLine.hasOption("partition")) { + int partition = + ((Number) commandLine.getParsedOptionValue("partition")).intValue(); + TopicPartition topicPartition = new TopicPartition(topic, partition); + zookeeperConnector.deleteCommittedOffsetPartitionCount(topicPartition); + } else { + zookeeperConnector.deleteCommittedOffsetTopicCount(topic); + } + } catch (Throwable t) { + LOG.error("Zookeeper client failed", t); + System.exit(1); + } + } +} diff --git a/src/main/java/com/pinterest/secor/message/Message.java b/src/main/java/com/pinterest/secor/message/Message.java new file mode 100644 index 000000000..c50ba9444 --- /dev/null +++ b/src/main/java/com/pinterest/secor/message/Message.java @@ -0,0 +1,74 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.message; + +import java.io.IOException; +import java.io.OutputStream; +import java.lang.String; + +/** + * Message represents a raw Kafka log message. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class Message { + private String mTopic; + private int mKafkaPartition; + private long mOffset; + private byte[] mPayload; + + protected String fieldsToString() { + return "topic='" + mTopic + '\'' + + ", kafkaPartition=" + mKafkaPartition + + ", offset=" + mOffset + + ", payload=" + new String(mPayload); + + } + + @Override + public String toString() { + return "Message{" + fieldsToString() + '}'; + } + + public Message(String topic, int kafkaPartition, long offset, byte[] payload) { + mTopic = topic; + mKafkaPartition = kafkaPartition; + mOffset = offset; + mPayload = payload; + } + + public String getTopic() { + + return mTopic; + } + + public int getKafkaPartition() { + return mKafkaPartition; + } + + public long getOffset() { + return mOffset; + } + + public byte[] getPayload() { + return mPayload; + } + + public void write(OutputStream output) throws IOException { + output.write(mPayload); + } +} diff --git a/src/main/java/com/pinterest/secor/message/ParsedMessage.java b/src/main/java/com/pinterest/secor/message/ParsedMessage.java new file mode 100644 index 000000000..47bec2b4c --- /dev/null +++ b/src/main/java/com/pinterest/secor/message/ParsedMessage.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.message; + +import java.lang.String; +import java.util.Arrays; + +/** + * Parsed message is a Kafka message that has been processed by the parser that extracted its + * partitions. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class ParsedMessage extends Message { + private String[] mPartitions; + + @Override + public String toString() { + return "ParsedMessage{" + fieldsToString() + ", mPartitions=" + + Arrays.toString(mPartitions) + '}'; + } + + public ParsedMessage(String topic, int kafkaPartition, long offset, byte[] payload, + String[] mPartitions) { + super(topic, kafkaPartition, offset, payload); + this.mPartitions = mPartitions; + } + + public String[] getPartitions() { + return mPartitions; + } +} diff --git a/src/main/java/com/pinterest/secor/parser/MessageParser.java b/src/main/java/com/pinterest/secor/parser/MessageParser.java new file mode 100644 index 000000000..b0770760e --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/MessageParser.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.parser; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; +import com.pinterest.secor.message.ParsedMessage; + +// TODO(pawel): should we offer a multi-message parser capable of parsing multiple types of +// messages? E.g., it could be implemented as a composite trying out different parsers and using +// the one that works. What is the performance cost of such approach? + +/** + * Message parser extracts partitions from messages. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public abstract class MessageParser { + protected SecorConfig mConfig; + + public MessageParser(SecorConfig config) { + mConfig = config; + } + + public ParsedMessage parse(Message message) throws Exception { + String[] partitions = extractPartitions(message); + return new ParsedMessage(message.getTopic(), message.getKafkaPartition(), + message.getOffset(), message.getPayload(), partitions); + } + + public abstract String[] extractPartitions(Message payload) throws Exception; +} diff --git a/src/main/java/com/pinterest/secor/parser/OffsetMessageParser.java b/src/main/java/com/pinterest/secor/parser/OffsetMessageParser.java new file mode 100644 index 000000000..ddb1c69e6 --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/OffsetMessageParser.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.parser; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; + +/** + * Offset message parser groups messages based on the offset ranges. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class OffsetMessageParser extends MessageParser { + public OffsetMessageParser(SecorConfig config) { + super(config); + } + + @Override + public String[] extractPartitions(Message message) throws Exception { + long offset = message.getOffset(); + long offsetsPerPartition = mConfig.getOffsetsPerPartition(); + long partition = (offset / offsetsPerPartition) * offsetsPerPartition; + String[] result = {"offset=" + partition}; + return result; + } +} diff --git a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java new file mode 100644 index 000000000..a64bace64 --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -0,0 +1,212 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.parser; + +import com.pinterest.secor.common.*; +import com.pinterest.secor.message.Message; +import com.pinterest.secor.util.FileUtil; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.*; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Topic finalizer writes _SUCCESS files to date partitions that very likely won't be receiving + * any new messages. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class PartitionFinalizer { + private static final Logger LOG = LoggerFactory.getLogger(PartitionFinalizer.class); + + private SecorConfig mConfig; + private ZookeeperConnector mZookeeperConnector; + private ThriftMessageParser mThriftMessageParser; + private KafkaClient mKafkaClient; + private QuboleClient mQuboleClient; + + public PartitionFinalizer(SecorConfig config) { + mConfig = config; + mKafkaClient = new KafkaClient(mConfig); + mZookeeperConnector = new ZookeeperConnector(mConfig); + mThriftMessageParser = new ThriftMessageParser(mConfig); + mQuboleClient = new QuboleClient(mConfig); + } + + private long getLastTimestampMillis(TopicPartition topicPartition) throws TException { + Message message = mKafkaClient.getLastMessage(topicPartition); + return mThriftMessageParser.extractTimestampMillis(message); + } + + private long getLastTimestampMillis(String topic) throws TException { + final int numPartitions = mKafkaClient.getNumPartitions(topic); + long max_timestamp = Long.MIN_VALUE; + for (int partition = 0; partition < numPartitions; ++partition) { + TopicPartition topicPartition = new TopicPartition(topic, partition); + long timestamp = getLastTimestampMillis(topicPartition); + if (timestamp > max_timestamp) { + max_timestamp = timestamp; + } + } + if (max_timestamp == Long.MIN_VALUE) { + return -1; + } + return max_timestamp; + } + + private long getCommittedTimestampMillis(TopicPartition topicPartition) throws Exception { + Message message = mKafkaClient.getCommittedMessage(topicPartition); + if (message == null) { + LOG.error("No message found for topic " + topicPartition.getTopic() + " partition " + + topicPartition.getPartition()); + return -1; + } + return mThriftMessageParser.extractTimestampMillis(message); + } + + private long getCommittedTimestampMillis(String topic) throws Exception { + final int numPartitions = mKafkaClient.getNumPartitions(topic); + long minTimestamp = Long.MAX_VALUE; + for (int partition = 0; partition < numPartitions; ++partition) { + TopicPartition topicPartition = new TopicPartition(topic, partition); + long timestamp = getCommittedTimestampMillis(topicPartition); + if (timestamp == -1) { + return -1; + } else { + if (timestamp < minTimestamp) { + minTimestamp = timestamp; + } + } + } + if (minTimestamp == Long.MAX_VALUE) { + return -1; + } + return minTimestamp; + } + + private NavigableSet getPartitions(String topic) throws IOException, ParseException { + final String s3Prefix = "s3n://" + mConfig.getS3Bucket() + "/" + mConfig.getS3Path(); + String[] partitions = {"dt="}; + LogFilePath logFilePath = new LogFilePath(s3Prefix, topic, partitions, + mConfig.getGeneration(), 0, 0); + String parentDir = logFilePath.getLogFileParentDir(); + String[] partitionDirs = FileUtil.list(parentDir); + Pattern pattern = Pattern.compile(".*/dt=(\\d\\d\\d\\d-\\d\\d-\\d\\d)$"); + TreeSet result = new TreeSet(); + for (String partitionDir : partitionDirs) { + Matcher matcher = pattern.matcher(partitionDir); + if (matcher.find()) { + String date = matcher.group(1); + SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd"); + format.setTimeZone(TimeZone.getTimeZone("UTC")); + Calendar calendar = Calendar.getInstance(TimeZone.getTimeZone("UTC")); + calendar.setTime(format.parse(date)); + result.add(calendar); + } + } + return result; + } + + private void finalizePartitionsUpTo(String topic, Calendar calendar) throws IOException, + ParseException, InterruptedException { + NavigableSet partitionDates = + getPartitions(topic).headSet(calendar, true).descendingSet(); + final String s3Prefix = "s3n://" + mConfig.getS3Bucket() + "/" + mConfig.getS3Path(); + SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd"); + format.setTimeZone(TimeZone.getTimeZone("UTC")); + for (Calendar partition : partitionDates) { + String partitionStr = format.format(partition.getTime()); + String[] partitions = {"dt=" + partitionStr}; + LogFilePath logFilePath = new LogFilePath(s3Prefix, topic, partitions, + mConfig.getGeneration(), 0, 0); + String logFileDir = logFilePath.getLogFileDir(); + assert FileUtil.exists(logFileDir) : "FileUtil.exists(" + logFileDir + ")"; + String successFilePath = logFileDir + "/_SUCCESS"; + if (FileUtil.exists(successFilePath)) { + return; + } + mQuboleClient.addPartition(topic, "dt='" + partitionStr + "'"); + LOG.info("touching file " + successFilePath); + FileUtil.touch(successFilePath); + } + } + + /** + * Get finalized timestamp for a given topic partition. Finalized timestamp is the current time + * if the last offset for that topic partition has been committed earlier than an hour ago. + * Otherwise, finalized timestamp is the committed timestamp. + * + * @param topicPartition The topic partition for which we want to compute the finalized + * timestamp. + * @return The finalized timestamp for the topic partition. + * @throws Exception + */ + private long getFinalizedTimestampMillis(TopicPartition topicPartition) throws Exception { + long lastTimestamp = getLastTimestampMillis(topicPartition); + long committedTimestamp = getCommittedTimestampMillis(topicPartition); + long now = System.currentTimeMillis(); + if (lastTimestamp == committedTimestamp && (now - lastTimestamp) > 3600 * 1000) { + return now; + } + return committedTimestamp; + } + + private long getFinalizedTimestampMillis(String topic) throws Exception { + final int numPartitions = mKafkaClient.getNumPartitions(topic); + long minTimestamp = Long.MAX_VALUE; + for (int partition = 0; partition < numPartitions; ++partition) { + TopicPartition topicPartition = new TopicPartition(topic, partition); + long timestamp = getFinalizedTimestampMillis(topicPartition); + LOG.info("finalized timestamp for topic " + topic + " partition " + partition + + " is " + timestamp); + if (timestamp == -1) { + return -1; + } else { + if (timestamp < minTimestamp) { + minTimestamp = timestamp; + } + } + } + if (minTimestamp == Long.MAX_VALUE) { + return -1; + } + return minTimestamp; + } + + public void finalizePartitions() throws Exception { + List topics = mZookeeperConnector.getCommittedOffsetTopics(); + for (String topic : topics) { + LOG.info("finalizing topic " + topic); + long finalizedTimestampMillis = getFinalizedTimestampMillis(topic); + LOG.info("finalized timestamp for topic " + topic + " is " + finalizedTimestampMillis); + if (finalizedTimestampMillis != -1) { + Calendar calendar = Calendar.getInstance(TimeZone.getTimeZone("UTC")); + calendar.setTimeInMillis(finalizedTimestampMillis); + // Introduce a lag of one day and one hour. + calendar.add(Calendar.HOUR, -1); + calendar.add(Calendar.DAY_OF_MONTH, -1); + finalizePartitionsUpTo(topic, calendar); + } + } + } +} diff --git a/src/main/java/com/pinterest/secor/parser/QuboleClient.java b/src/main/java/com/pinterest/secor/parser/QuboleClient.java new file mode 100644 index 000000000..e39228671 --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/QuboleClient.java @@ -0,0 +1,112 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.parser; + +import com.pinterest.secor.common.SecorConfig; +import net.minidev.json.JSONObject; +import net.minidev.json.JSONValue; + +import java.io.*; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.Map; + +/** + * Qubole client encapsulates communication with a Qubole cluster. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class QuboleClient { + private String mApiToken; + + public QuboleClient(SecorConfig config) { + mApiToken = config.getQuboleApiToken(); + } + + private Map makeRequest(URL url, String body) throws IOException { + HttpURLConnection connection = null; + try { + connection = (HttpURLConnection) url.openConnection(); + connection.setRequestProperty("X-AUTH-TOKEN", mApiToken); + connection.setRequestProperty("Content-Type", "application/json"); + connection.setRequestProperty("Accepts", "application/json"); + connection.setRequestProperty("Accept", "*/*"); + if (body != null) { + connection.setRequestMethod("POST"); + connection.setRequestProperty("Content-Length", + Integer.toString(body.getBytes().length)); + } + connection.setUseCaches (false); + connection.setDoInput(true); + connection.setDoOutput(true); + + if (body != null) { + // Send request. + DataOutputStream dataOutputStream = new DataOutputStream( + connection.getOutputStream()); + dataOutputStream.writeBytes(body); + dataOutputStream.flush(); + dataOutputStream.close(); + } + + // Get Response. + InputStream inputStream = connection.getInputStream(); + BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream)); + Map response = (Map) JSONValue.parse(reader); + if (response.get("status").equals("error")) { + throw new RuntimeException("command " + url + " with body " + body + " failed " + + JSONObject.toJSONString(response)); + } + return response; + } catch (IOException exception) { + if (connection != null) { + connection.disconnect(); + } + throw exception; + } + } + + private int query(String query) throws IOException { + URL url = new URL("http://api.qubole.com/api/v1.2/commands"); + JSONObject queryJson = new JSONObject(); + queryJson.put("query", query); + String body = queryJson.toString(); + Map response = makeRequest(url, body); + return (Integer) response.get("id"); + } + + private void waitForCompletion(int commandId) throws IOException, InterruptedException { + URL url = new URL("http://api.qubole.com/api/v1.2/commands/" + commandId); + while (true) { + Map response = makeRequest(url, null); + if (response.get("status").equals("done")) { + return; + } + System.out.println("waiting 3 seconds for results of query " + commandId + + ". Current status " + response.get("status")); + Thread.sleep(3000); + } + } + + public void addPartition(String table, String partition) throws IOException, + InterruptedException { + String queryStr = "ALTER TABLE " + table + " ADD IF NOT EXISTS PARTITION (" + partition + + ")"; + int commandId = query(queryStr); + waitForCompletion(commandId); + } +} diff --git a/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java b/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java new file mode 100644 index 000000000..da8ae9e12 --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java @@ -0,0 +1,83 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.parser; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; +import org.apache.thrift.TDeserializer; +import org.apache.thrift.TException; +import org.apache.thrift.TFieldIdEnum; + +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.TimeZone; + +/** + * Thrift message parser extracts date partitions from thrift messages. + * TODO(pawel): implement. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class ThriftMessageParser extends MessageParser { + private TDeserializer mDeserializer; + + public ThriftMessageParser(SecorConfig config) { + super(config); + mDeserializer = new TDeserializer(); + } + + public long extractTimestampMillis(Message message) throws TException { + class ThriftTemplate implements TFieldIdEnum { + public ThriftTemplate() { + } + + @Override + public short getThriftFieldId() { + return 1; + } + + @Override + public String getFieldName() { + return "timestamp"; + } + } + long timestamp = mDeserializer.partialDeserializeI64(message.getPayload(), + new ThriftTemplate()); + final long nanosecondDivider = (long) Math.pow(10, 9 + 9); + final long millisecondDivider = (long) Math.pow(10, 9 + 3); + long timestampMillis; + if (timestamp / nanosecondDivider > 0L) { + timestampMillis = timestamp / (long) Math.pow(10, 6); + } else if (timestamp / millisecondDivider > 0L) { + timestampMillis = timestamp; + } else { // assume seconds + timestampMillis = timestamp * 1000L; + } + return timestampMillis; + } + + @Override + public String[] extractPartitions(Message message) throws Exception { + // Date constructor takes milliseconds since epoch. + long timestampMillis = extractTimestampMillis(message); + Date date = new Date(timestampMillis); + SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd"); + format.setTimeZone(TimeZone.getTimeZone("UTC")); + String[] result = {"dt=" + format.format(date)}; + return result; + } +} diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java new file mode 100644 index 000000000..c656bcdde --- /dev/null +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -0,0 +1,133 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.reader; + +import com.pinterest.secor.common.OffsetTracker; +import com.pinterest.secor.common.TopicPartition; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; +import com.pinterest.secor.util.IdUtil; +import com.pinterest.secor.util.RateLimitUtil; +import com.pinterest.secor.util.StatsUtil; +import kafka.consumer.*; +import kafka.javaapi.consumer.ConsumerConnector; + +import java.net.UnknownHostException; +import java.util.*; + +import kafka.message.MessageAndMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Message reader consumer raw Kafka messages. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class MessageReader { + private static final Logger LOG = LoggerFactory.getLogger(MessageReader.class); + + private SecorConfig mConfig; + private OffsetTracker mOffsetTracker; + private ConsumerConnector mConsumerConnector; + private ConsumerIterator mIterator; + private HashMap mLastAccessTime; + + public MessageReader(SecorConfig config, OffsetTracker offsetTracker) throws + UnknownHostException { + mConfig = config; + mOffsetTracker = offsetTracker; + + mConsumerConnector = Consumer.createJavaConsumerConnector(createConsumerConfig()); + + TopicFilter topicFilter = new Whitelist(mConfig.getKafkaTopicFilter()); + List> streams = + mConsumerConnector.createMessageStreamsByFilter(topicFilter); + KafkaStream stream = streams.get(0); + mIterator = stream.iterator(); + mLastAccessTime = new HashMap(); + StatsUtil.setLabel("secor.kafka.consumer.id", IdUtil.getConsumerId()); + } + + private void updateAccessTime(TopicPartition topicPartition) { + long now = System.currentTimeMillis() / 1000L; + mLastAccessTime.put(topicPartition, now); + Iterator iterator = mLastAccessTime.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry pair = (Map.Entry) iterator.next(); + long lastAccessTime = (Long) pair.getValue(); + if (now - lastAccessTime > mConfig.getTopicPartitionForgetSeconds()) { + iterator.remove(); + } + } + } + + private void exportStats() { + StringBuffer topicPartitions = new StringBuffer(); + for (TopicPartition topicPartition : mLastAccessTime.keySet()) { + if (topicPartitions.length() > 0) { + topicPartitions.append(' '); + } + topicPartitions.append(topicPartition.getTopic() + '/' + + topicPartition.getPartition()); + } + StatsUtil.setLabel("secor.topic_partitions", topicPartitions.toString()); + } + + private ConsumerConfig createConsumerConfig() throws UnknownHostException { + Properties props = new Properties(); + props.put("zookeeper.connect", mConfig.getZookeeperQuorum()); + props.put("group.id", mConfig.getKafkaGroup()); + + props.put("zookeeper.session.timeout.ms", + Integer.toString(mConfig.getZookeeperSessionTimeoutMs())); + props.put("zookeeper.sync.time.ms", Integer.toString(mConfig.getZookeeperSyncTimeMs())); + props.put("auto.commit.enable", "false"); + // This option is required to make sure that messages are not lost for new topics and + // topics whose number of partitions has changed. + props.put("auto.offset.reset", "smallest"); + props.put("consumer.timeout.ms", Integer.toString(mConfig.getConsumerTimeoutMs())); + props.put("consumer.id", IdUtil.getConsumerId()); + + return new ConsumerConfig(props); + } + + public boolean hasNext() { + return mIterator.hasNext(); + } + + public Message read() { + assert hasNext(); + RateLimitUtil.acquire(); + MessageAndMetadata kafkaMessage = mIterator.next(); + Message message = new Message(kafkaMessage.topic(), kafkaMessage.partition(), + kafkaMessage.offset(), kafkaMessage.message()); + TopicPartition topicPartition = new TopicPartition(message.getTopic(), + message.getKafkaPartition()); + updateAccessTime(topicPartition); + // Skip already committed messages. + long committedOffsetCount = mOffsetTracker.getTrueCommittedOffsetCount(topicPartition); + LOG.debug("read message" + message); + exportStats(); + if (message.getOffset() < committedOffsetCount) { + LOG.debug("skipping message message " + message + " because its offset precedes " + + "committed offset count " + committedOffsetCount); + return null; + } + return message; + } +} diff --git a/src/main/java/com/pinterest/secor/tools/LogFilePrinter.java b/src/main/java/com/pinterest/secor/tools/LogFilePrinter.java new file mode 100644 index 000000000..e4dc9c15d --- /dev/null +++ b/src/main/java/com/pinterest/secor/tools/LogFilePrinter.java @@ -0,0 +1,63 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.tools; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.util.FileUtil; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.SequenceFile; + +import java.io.IOException; + +/** + * Log file printer displays the content of a log file. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class LogFilePrinter { + private SecorConfig mConfig; + private FileSystem mFileSystem; + private boolean mPrintOffsetsOnly; + + public LogFilePrinter(SecorConfig config, boolean printOffsetsOnly) throws IOException { + mConfig = config; + Configuration configuration = new Configuration(); + mFileSystem = FileSystem.get(configuration); + mPrintOffsetsOnly = printOffsetsOnly; + } + + public void printFile(String path) throws Exception { + FileSystem fileSystem = FileUtil.getFileSystem(path); + Path fsPath = new Path(path); + SequenceFile.Reader reader = new SequenceFile.Reader(fileSystem, fsPath, + new Configuration()); + LongWritable key = (LongWritable) reader.getKeyClass().newInstance(); + BytesWritable value = (BytesWritable) reader.getValueClass().newInstance(); + System.out.println("reading file " + path); + while (reader.next(key, value)) { + if (mPrintOffsetsOnly) { + System.out.println(Long.toString(key.get())); + } else { + System.out.println(Long.toString(key.get()) + ": " + new String(value.getBytes())); + } + } + } +} diff --git a/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java b/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java new file mode 100644 index 000000000..681374d1b --- /dev/null +++ b/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java @@ -0,0 +1,208 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.tools; + +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.common.TopicPartition; +import com.pinterest.secor.util.FileUtil; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.SequenceFile; + +import java.io.IOException; +import java.util.*; + +/** + * Log file verifier checks the consistency of log files. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class LogFileVerifier { + private SecorConfig mConfig; + private String mTopic; + private HashMap>> + mTopicPartitionToOffsetToFiles; + + public LogFileVerifier(SecorConfig config, String topic) throws IOException { + mConfig = config; + mTopic = topic; + mTopicPartitionToOffsetToFiles = + new HashMap>>(); + } + + private String getPrefix() { + return "s3n://" + mConfig.getS3Bucket() + "/" + mConfig.getS3Path(); + } + + private String getTopicPrefix() { + return getPrefix() + "/" + mTopic; + } + + private void populateTopicPartitionToOffsetToFiles() throws IOException { + String prefix = getPrefix(); + String topicPrefix = getTopicPrefix(); + String[] paths = FileUtil.listRecursively(topicPrefix); + for (String path : paths) { + if (!path.endsWith("/_SUCCESS")) { + LogFilePath logFilePath = new LogFilePath(prefix, path); + TopicPartition topicPartition = new TopicPartition(logFilePath.getTopic(), + logFilePath.getKafkaPartition()); + SortedMap> offsetToFiles = + mTopicPartitionToOffsetToFiles.get(topicPartition); + if (offsetToFiles == null) { + offsetToFiles = new TreeMap>(); + mTopicPartitionToOffsetToFiles.put(topicPartition, offsetToFiles); + } + long offset = logFilePath.getOffset(); + HashSet logFilePaths = offsetToFiles.get(offset); + if (logFilePaths == null) { + logFilePaths = new HashSet(); + offsetToFiles.put(offset, logFilePaths); + } + logFilePaths.add(logFilePath); + } + } + } + + private void filterOffsets(long fromOffset, long toOffset) { + Iterator iterator = mTopicPartitionToOffsetToFiles.entrySet().iterator(); + while (iterator.hasNext()) { + long firstOffset = -2; + long lastOffset = Long.MAX_VALUE; + Map.Entry entry = (Map.Entry) iterator.next(); + SortedMap> offsetToFiles = + (SortedMap>) entry.getValue(); + for (long offset : offsetToFiles.keySet()) { + if (offset <= fromOffset || firstOffset == -2) { + firstOffset = offset; + } + if (offset >= toOffset && toOffset == Long.MAX_VALUE) { + lastOffset = offset; + } + } + if (firstOffset != -2) { + TopicPartition topicPartition = (TopicPartition) entry.getKey(); + offsetToFiles = offsetToFiles.subMap(firstOffset, lastOffset); + mTopicPartitionToOffsetToFiles.put(topicPartition, offsetToFiles); + } + } + } + + private int getMessageCount(LogFilePath logFilePath) throws Exception { + String path = logFilePath.getLogFilePath(); + Path fsPath = new Path(path); + FileSystem fileSystem = FileUtil.getFileSystem(path); + SequenceFile.Reader reader = new SequenceFile.Reader(fileSystem, fsPath, + new Configuration()); + LongWritable key = (LongWritable) reader.getKeyClass().newInstance(); + BytesWritable value = (BytesWritable) reader.getValueClass().newInstance(); + int result = 0; + while (reader.next(key, value)) { + result++; + } + reader.close(); + return result; + } + + public void verifyCounts(long fromOffset, long toOffset, int numMessages) throws Exception { + populateTopicPartitionToOffsetToFiles(); + filterOffsets(fromOffset, toOffset); + Iterator iterator = mTopicPartitionToOffsetToFiles.entrySet().iterator(); + int aggregateMessageCount = 0; + while (iterator.hasNext()) { + long previousOffset = -2L; + long previousMessageCount = -2L; + Map.Entry entry = (Map.Entry) iterator.next(); + SortedMap> offsetToFiles = + (SortedMap>) entry.getValue(); + for (HashSet logFilePaths : offsetToFiles.values()) { + int messageCount = 0; + long offset = -2; + for (LogFilePath logFilePath : logFilePaths) { + assert offset == -2 || offset == logFilePath.getOffset(): + Long.toString(offset) + " || " + offset + " == " + logFilePath.getOffset(); + messageCount += getMessageCount(logFilePath); + offset = logFilePath.getOffset(); + } + if (previousOffset != -2 && offset - previousOffset != previousMessageCount) { + TopicPartition topicPartition = (TopicPartition) entry.getKey(); + throw new RuntimeException("Message count of " + previousMessageCount + + " in topic " + topicPartition.getTopic() + + " partition " + topicPartition.getPartition() + + " does not agree with adjacent offsets " + + previousOffset + " and " + offset); + } + previousOffset = offset; + previousMessageCount = messageCount; + aggregateMessageCount += messageCount; + } + } + if (numMessages != -1 && aggregateMessageCount != numMessages) { + throw new RuntimeException("Message count " + aggregateMessageCount + + " does not agree with the expected count " + numMessages); + } + } + + private void getOffsets(LogFilePath logFilePath, Set offsets) throws Exception { + String path = logFilePath.getLogFilePath(); + Path fsPath = new Path(path); + FileSystem fileSystem = FileUtil.getFileSystem(path); + SequenceFile.Reader reader = new SequenceFile.Reader(fileSystem, fsPath, + new Configuration()); + LongWritable key = (LongWritable) reader.getKeyClass().newInstance(); + BytesWritable value = (BytesWritable) reader.getValueClass().newInstance(); + while (reader.next(key, value)) { + if (!offsets.add(key.get())) { + throw new RuntimeException("duplicate key " + key.get() + " found in file " + + logFilePath.getLogFilePath()); + } + } + reader.close(); + } + + public void verifySequences(long fromOffset, long toOffset) throws Exception { + populateTopicPartitionToOffsetToFiles(); + filterOffsets(fromOffset, toOffset); + + Iterator iterator = mTopicPartitionToOffsetToFiles.entrySet().iterator(); + while (iterator.hasNext()) { + TreeSet offsets = new TreeSet(); + Map.Entry entry = (Map.Entry) iterator.next(); + TopicPartition topicPartition = (TopicPartition) entry.getKey(); + SortedMap> offsetToFiles = + (SortedMap>) entry.getValue(); + for (HashSet logFilePaths : offsetToFiles.values()) { + for (LogFilePath logFilePath : logFilePaths) { + getOffsets(logFilePath, offsets); + } + } + long lastOffset = -2; + for (Long offset : offsets) { + if (lastOffset != -2) { + assert lastOffset + 1 == offset: Long.toString(offset) + " + 1 == " + offset + + " for topic " + topicPartition.getTopic() + " partition " + + topicPartition.getPartition(); + } + lastOffset = offset; + } + } + } +} diff --git a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java new file mode 100644 index 000000000..9bf0210dd --- /dev/null +++ b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java @@ -0,0 +1,157 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.tools; + +import com.pinterest.secor.common.KafkaClient; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.common.TopicPartition; +import com.pinterest.secor.common.ZookeeperConnector; +import com.pinterest.secor.message.Message; +import com.pinterest.secor.parser.ThriftMessageParser; +import com.pinterest.secor.util.StatsUtil; +import net.minidev.json.JSONObject; +import net.minidev.json.JSONValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.*; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Progress monitor exports offset lags par topic partition. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class ProgressMonitor { + private static final Logger LOG = LoggerFactory.getLogger(ProgressMonitor.class); + private SecorConfig mConfig; + private ZookeeperConnector mZookeeperConnector; + private KafkaClient mKafkaClient; + private ThriftMessageParser mThriftMessageParser; + + public ProgressMonitor(SecorConfig config) { + mConfig = config; + mZookeeperConnector = new ZookeeperConnector(mConfig); + mKafkaClient = new KafkaClient(mConfig); + mThriftMessageParser = new ThriftMessageParser(mConfig); + } + + private void makeRequest(String body) throws IOException { + URL url = new URL("http://" + mConfig.getTsdbHostport() + "/api/put?details"); + HttpURLConnection connection = null; + try { + connection = (HttpURLConnection) url.openConnection(); + connection.setRequestProperty("Content-Type", "application/json"); + connection.setRequestProperty("Accepts", "application/json"); + connection.setRequestProperty("Accept", "*/*"); + if (body != null) { + connection.setRequestMethod("POST"); + connection.setRequestProperty("Content-Length", + Integer.toString(body.getBytes().length)); + } + connection.setUseCaches (false); + connection.setDoInput(true); + connection.setDoOutput(true); + + if (body != null) { + // Send request. + DataOutputStream dataOutputStream = new DataOutputStream( + connection.getOutputStream()); + dataOutputStream.writeBytes(body); + dataOutputStream.flush(); + dataOutputStream.close(); + } + + // Get Response. + InputStream inputStream = connection.getInputStream(); + BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream)); + Map response = (Map) JSONValue.parse(reader); + if (!response.get("failed").equals(0)) { + throw new RuntimeException("url " + url + " with body " + body + " failed " + + JSONObject.toJSONString(response)); + } + } catch (IOException exception) { + if (connection != null) { + connection.disconnect(); + } + throw exception; + } + } + + private void exportToTsdb(String metric, Map tags, String value) + throws IOException { + JSONObject bodyJson = new JSONObject(); + bodyJson.put("metric", metric); + bodyJson.put("timestamp", System.currentTimeMillis() / 1000); + bodyJson.put("value", value); + JSONObject tagsJson = new JSONObject(); + for (Map.Entry entry : tags.entrySet()) { + tagsJson.put(entry.getKey(), entry.getValue()); + } + bodyJson.put("tags", tagsJson); + LOG.info("exporting metric to tsdb " + bodyJson); + makeRequest(bodyJson.toString()); + } + + public void exportStats() throws Exception { + List topics = mZookeeperConnector.getCommittedOffsetTopics(); + for (String topic : topics) { + List partitions = mZookeeperConnector.getCommittedOffsetPartitions(topic); + for (Integer partition : partitions) { + TopicPartition topicPartition = new TopicPartition(topic, partition); + Message committedMessage = mKafkaClient.getCommittedMessage(topicPartition); + long committedOffset = - 1; + long committedTimestampMillis = -1; + if (committedMessage == null) { + LOG.warn("no committed message found in topic " + topic + " partition " + + partition); + } else { + committedOffset = committedMessage.getOffset(); + committedTimestampMillis = mThriftMessageParser.extractTimestampMillis( + committedMessage); + } + + Message lastMessage = mKafkaClient.getLastMessage(topicPartition); + if (lastMessage == null) { + LOG.warn("no message found in topic " + topic + " partition " + partition); + } else { + long lastOffset = lastMessage.getOffset(); + long lastTimestampMillis = mThriftMessageParser.extractTimestampMillis( + lastMessage); + assert committedOffset <= lastOffset: Long.toString(committedOffset) + " <= " + + lastOffset; + long offsetLag = lastOffset - committedOffset; + long timestampMillisLag = lastTimestampMillis - committedTimestampMillis; + HashMap tags = new HashMap(); + tags.put("topic", topic); + tags.put("partition", Integer.toString(partition)); + exportToTsdb("secor.lag.offsets", tags, Long.toString(offsetLag)); + exportToTsdb("secor.lag.seconds", tags, + Long.toString(timestampMillisLag / 1000)); + LOG.debug("topic " + topic + " partition " + partition + " committed offset " + + committedOffset + " last offset " + lastOffset + " committed timestamp " + + (committedTimestampMillis / 1000) + " last timestamp " + + (lastTimestampMillis / 1000)); + } + } + } + } +} diff --git a/src/main/java/com/pinterest/secor/tools/RandomPartitioner.java b/src/main/java/com/pinterest/secor/tools/RandomPartitioner.java new file mode 100644 index 000000000..1c8d2ffcf --- /dev/null +++ b/src/main/java/com/pinterest/secor/tools/RandomPartitioner.java @@ -0,0 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.tools; + +import kafka.producer.Partitioner; +import kafka.utils.VerifiableProperties; + +/** + * Random partitioner spreads messages evenly across partitions. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class RandomPartitioner implements Partitioner { + public RandomPartitioner(VerifiableProperties properties) { + } + + public int partition(String key, int numPartitions) { + int intKey = Integer.parseInt(key); + return intKey % numPartitions; + } +} diff --git a/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java b/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java new file mode 100644 index 000000000..e85143b3b --- /dev/null +++ b/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java @@ -0,0 +1,76 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.tools; + +import kafka.javaapi.producer.Producer; +import kafka.producer.KeyedMessage; +import kafka.producer.ProducerConfig; +import org.apache.thrift.TException; +import org.apache.thrift.TSerializer; +import org.apache.thrift.protocol.TBinaryProtocol; +import com.pinterest.secor.thrift.TestMessage; +import com.pinterest.secor.thrift.TestEnum; + +import java.util.Properties; + +/** + * Test log message producer generates test messages and submits them to kafka. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class TestLogMessageProducer extends Thread { + private String mTopic; + private int mNumMessages; + + public TestLogMessageProducer(String topic, int numMessages) { + mTopic = topic; + mNumMessages = numMessages; + } + + public void run() { + Properties properties = new Properties(); + properties.put("metadata.broker.list", "localhost:9092"); + properties.put("partitioner.class", "com.pinterest.secor.tools.RandomPartitioner"); + properties.put("serializer.class", "kafka.serializer.DefaultEncoder"); + properties.put("key.serializer.class", "kafka.serializer.StringEncoder"); + properties.put("request.required.acks", "1"); + + ProducerConfig config = new ProducerConfig(properties); + Producer producer = new Producer(config); + + TSerializer serializer = new TSerializer(new TBinaryProtocol.Factory()); + for (int i = 0; i < mNumMessages; ++i) { + TestMessage testMessage = new TestMessage(System.currentTimeMillis() * 1000000L + i, + "some_value_" + i); + if (i % 2 == 0) { + testMessage.setEnumField(TestEnum.SOME_VALUE); + } else { + testMessage.setEnumField(TestEnum.SOME_OTHER_VALUE); + } + byte[] bytes; + try { + bytes = serializer.serialize(testMessage); + } catch(TException e) { + throw new RuntimeException("Failed to serialize message " + testMessage, e); + } + KeyedMessage data = new KeyedMessage( + mTopic, Integer.toString(i), bytes); + producer.send(data); + } + producer.close(); + } +} diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java new file mode 100644 index 000000000..22ac4b336 --- /dev/null +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -0,0 +1,199 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.uploader; + +import com.pinterest.secor.common.*; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.util.FileUtil; +import com.pinterest.secor.util.IdUtil; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.*; + +/** + * Uploader applies a set of policies to determine if any of the locally stored files should be + * uploaded to s3. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class Uploader { + private static final Logger LOG = LoggerFactory.getLogger(Uploader.class); + + private SecorConfig mConfig; + private OffsetTracker mOffsetTracker; + private FileRegistry mFileRegistry; + private ZookeeperConnector mZookeeperConnector; + + public Uploader(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry) { + this(config, offsetTracker, fileRegistry, new ZookeeperConnector(config)); + } + + // For testing use only. + public Uploader(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry, + ZookeeperConnector zookeeperConnector) { + mConfig = config; + mOffsetTracker = offsetTracker; + mFileRegistry = fileRegistry; + mZookeeperConnector = zookeeperConnector; + } + + private void upload(LogFilePath localPath) throws Exception { + String s3Prefix = "s3n://" + mConfig.getS3Bucket() + "/" + mConfig.getS3Path(); + LogFilePath s3Path = new LogFilePath(s3Prefix, localPath.getTopic(), + localPath.getPartitions(), + localPath.getGeneration(), + localPath.getKafkaPartition(), + localPath.getOffset()); + String localLogFilename = localPath.getLogFilePath(); + LOG.info("uploading file " + localLogFilename + " to " + s3Path.getLogFilePath()); + FileUtil.moveToS3(localLogFilename, s3Path.getLogFilePath()); + } + + private void uploadFiles(TopicPartition topicPartition) throws Exception { + long committedOffsetCount = mOffsetTracker.getTrueCommittedOffsetCount(topicPartition); + long lastSeenOffset = mOffsetTracker.getLastSeenOffset(topicPartition); + final String lockPath = "/secor/locks/" + topicPartition.getTopic() + "/" + + topicPartition.getPartition(); + // Deleting writers closes their streams flushing all pending data to the disk. + mFileRegistry.deleteWriters(topicPartition); + mZookeeperConnector.lock(lockPath); + try { + // Check if the committed offset has changed. + long zookeeperComittedOffsetCount = mZookeeperConnector.getCommittedOffsetCount( + topicPartition); + if (zookeeperComittedOffsetCount == committedOffsetCount) { + LOG.info("uploading topic " + topicPartition.getTopic() + " partition " + + topicPartition.getPartition()); + Collection paths = mFileRegistry.getPaths(topicPartition); + for (LogFilePath path : paths) { + upload(path); + } + mFileRegistry.deleteTopicPartition(topicPartition); + mZookeeperConnector.setCommittedOffsetCount(topicPartition, lastSeenOffset + 1); + mOffsetTracker.setCommittedOffsetCount(topicPartition, lastSeenOffset + 1); + } + } finally { + mZookeeperConnector.unlock(lockPath); + } + } + + /** + * This method is intended to be overwritten in tests. + */ + protected SequenceFile.Reader createReader(FileSystem fileSystem, Path path, + Configuration configuration) throws IOException { + return new SequenceFile.Reader(fileSystem, path, configuration); + } + + private void trim(LogFilePath srcPath, long startOffset) throws Exception { + if (startOffset == srcPath.getOffset()) { + return; + } + Configuration config = new Configuration(); + FileSystem fs = FileSystem.get(config); + String srcFilename = srcPath.getLogFilePath(); + Path srcFsPath = new Path(srcFilename); + SequenceFile.Reader reader = null; + SequenceFile.Writer writer = null; + LogFilePath dstPath = null; + int copiedMessages = 0; + // Deleting the writer closes its stream flushing all pending data to the disk. + mFileRegistry.deleteWriter(srcPath); + try { + reader = createReader(fs, srcFsPath, config); + LongWritable key = (LongWritable) reader.getKeyClass().newInstance(); + BytesWritable value = (BytesWritable) reader.getValueClass().newInstance(); + while (reader.next(key, value)) { + if (key.get() >= startOffset) { + if (writer == null) { + String localPrefix = mConfig.getLocalPath() + '/' + + IdUtil.getLocalMessageDir(); + dstPath = new LogFilePath(localPrefix, srcPath.getTopic(), + srcPath.getPartitions(), srcPath.getGeneration(), + srcPath.getKafkaPartition(), startOffset); + writer = mFileRegistry.getOrCreateWriter(dstPath); + } + writer.append(key, value); + copiedMessages++; + } + } + } finally { + if (reader != null) { + reader.close(); + } + } + mFileRegistry.deletePath(srcPath); + if (dstPath == null) { + LOG.info("removed file " + srcPath.getLogFilePath()); + } else { + LOG.info("trimmed " + copiedMessages + " messages from " + srcFilename + " to " + + dstPath.getLogFilePath() + " with start offset " + startOffset); + } + } + + private void trimFiles(TopicPartition topicPartition, long startOffset) throws Exception { + Collection paths = mFileRegistry.getPaths(topicPartition); + for (LogFilePath path : paths) { + trim(path, startOffset); + } + } + + private void checkTopicPartition(TopicPartition topicPartition) throws Exception { + final long size = mFileRegistry.getSize(topicPartition); + final long modificationAgeSec = mFileRegistry.getModificationAgeSec(topicPartition); + if (size >= mConfig.getMaxFileSizeBytes() || + modificationAgeSec >= mConfig.getMaxFileAgeSeconds()) { + long newOffsetCount = mZookeeperConnector.getCommittedOffsetCount(topicPartition); + long oldOffsetCount = mOffsetTracker.setCommittedOffsetCount(topicPartition, + newOffsetCount); + long lastSeenOffset = mOffsetTracker.getLastSeenOffset(topicPartition); + if (oldOffsetCount == newOffsetCount) { + uploadFiles(topicPartition); + } else if (newOffsetCount > lastSeenOffset) { // && oldOffset < newOffset + LOG.debug("last seen offset " + lastSeenOffset + + " is lower than committed offset count " + newOffsetCount + + ". Deleting files in topic " + topicPartition.getTopic() + + " partition " + topicPartition.getPartition()); + // There was a rebalancing event and someone committed an offset beyond that of the + // current message. We need to delete the local file. + mFileRegistry.deleteTopicPartition(topicPartition); + } else { // oldOffsetCount < newOffsetCount <= lastSeenOffset + LOG.debug("previous committed offset count " + oldOffsetCount + + " is lower than committed offset " + newOffsetCount + + " is lower than or equal to last seen offset " + lastSeenOffset + + ". Trimming files in topic " + topicPartition.getTopic() + + " partition " + topicPartition.getPartition()); + // There was a rebalancing event and someone committed an offset lower than that + // of the current message. We need to trim local files. + trimFiles(topicPartition, newOffsetCount); + } + } + } + + public void applyPolicy() throws Exception { + Collection topicPartitions = mFileRegistry.getTopicPartitions(); + for (TopicPartition topicPartition : topicPartitions) { + checkTopicPartition(topicPartition); + } + } +} diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java new file mode 100644 index 000000000..2362c2ea8 --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -0,0 +1,114 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.util; + +import com.pinterest.secor.common.SecorConfig; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URI; +import java.util.ArrayList; +import java.util.Arrays; + +/** + * File util implements utilities for interactions with the file system. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class FileUtil { + private static final Logger LOG = LoggerFactory.getLogger(FileUtil.class); + private static SecorConfig mConfig = null; + + public static void configure(SecorConfig config) { + mConfig = config; + } + + public static FileSystem getFileSystem(String path) throws IOException { + Configuration conf = new Configuration(); + if (mConfig != null) { + conf.set("fs.s3n.awsAccessKeyId", mConfig.getAwsAccessKey()); + conf.set("fs.s3n.awsSecretAccessKey", mConfig.getAwsSecretKey()); + } + return FileSystem.get(URI.create(path), conf); + } + + public static String[] list(String path) throws IOException { + FileSystem fs = getFileSystem(path); + Path fsPath = new Path(path); + ArrayList paths = new ArrayList(); + FileStatus[] statuses = fs.listStatus(fsPath); + if (statuses != null) { + for (FileStatus status : statuses) { + Path statusPath = status.getPath(); + if (path.startsWith("s3://") || path.startsWith("s3n://")) { + paths.add(statusPath.toUri().toString()); + } else { + paths.add(statusPath.toUri().getPath()); + } + } + } + return paths.toArray(new String[] {}); + } + + public static String[] listRecursively(String path) throws IOException { + ArrayList paths = new ArrayList(); + String[] directPaths = list(path); + for (String directPath : directPaths) { + if (directPath.equals(path)) { + assert directPaths.length == 1: Integer.toString(directPaths.length) + " == 1"; + paths.add(directPath); + } else { + String[] recursivePaths = listRecursively(directPath); + paths.addAll(Arrays.asList(recursivePaths)); + } + } + return paths.toArray(new String[] {}); + } + + public static boolean exists(String path) throws IOException { + FileSystem fs = getFileSystem(path); + Path fsPath = new Path(path); + return fs.exists(fsPath); + } + + public static void delete(String path) throws IOException { + if (exists(path)) { + Path fsPath = new Path(path); + boolean success = getFileSystem(path).delete(fsPath, true); // recursive + if (!success) { + throw new IOException("Failed to delete " + path); + } + } + } + + public static void moveToS3(String srcLocalPath, String dstS3Path) throws IOException { + Path srcPath = new Path(srcLocalPath); + Path dstPath = new Path(dstS3Path); + getFileSystem(dstS3Path).moveFromLocalFile(srcPath, dstPath); + } + + public static void touch(String path) throws IOException { + FileSystem fs = getFileSystem(path); + Path fsPath = new Path(path); + fs.create(fsPath).close(); + } +} diff --git a/src/main/java/com/pinterest/secor/util/IdUtil.java b/src/main/java/com/pinterest/secor/util/IdUtil.java new file mode 100644 index 000000000..122be30ff --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/IdUtil.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.util; + +import java.lang.management.ManagementFactory; +import java.net.InetAddress; +import java.net.UnknownHostException; + +/** + * Utilities related to identifiers. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class IdUtil { + public static String getConsumerId() throws UnknownHostException { + String hostname = InetAddress.getLocalHost().getHostName(); + String pid = ManagementFactory.getRuntimeMXBean().getName().split("@")[0]; + long threadId = Thread.currentThread().getId(); + return hostname + "_" + pid + "_" + threadId; + } + + public static String getLocalMessageDir() throws UnknownHostException { + String pid = ManagementFactory.getRuntimeMXBean().getName().split("@")[0]; + long threadId = Thread.currentThread().getId(); + return pid + "_" + threadId; + } +} diff --git a/src/main/java/com/pinterest/secor/util/RateLimitUtil.java b/src/main/java/com/pinterest/secor/util/RateLimitUtil.java new file mode 100644 index 000000000..57cb4b5c9 --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/RateLimitUtil.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.util; + +import com.google.common.util.concurrent.RateLimiter; +import com.pinterest.secor.common.SecorConfig; + +/** + * Rate limit util wraps around a rate limiter shared across consumer threads. The rate limiting + * mechanism does not prevent temporary bursts of the load on the broker caused by the message + * prefetching mechanism of the native kafka client. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class RateLimitUtil { + private static RateLimiter mRateLimiter = null; + + public static void configure(SecorConfig config) { + // Lazy initialization of the rate limiter would have to be in a synchronized block so + // creating it here makes things simple. + mRateLimiter = RateLimiter.create(config.getMessagesPerSecond()); + } + + public static void acquire() { + mRateLimiter.acquire(); + } +} diff --git a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java new file mode 100644 index 000000000..071a5ffef --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.util; + +import com.pinterest.secor.common.SecorConfig; + +import java.lang.reflect.Constructor; + +/** + * ReflectionUtil implements utility methods to construct objects of classes specified by name. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class ReflectionUtil { + public static Object createMessageParser(String className, + SecorConfig config) throws Exception { + Class clazz = Class.forName(className); + + // Search for an "appropriate" constructor. + for (Constructor ctor : clazz.getConstructors()) { + Class[] paramTypes = ctor.getParameterTypes(); + + // If the arity matches, let's use it. + if (paramTypes.length == 1) { + Object[] args = {config}; + return ctor.newInstance(args); + } + } + throw new IllegalArgumentException("Class not found " + className); + } +} diff --git a/src/main/java/com/pinterest/secor/util/StatsUtil.java b/src/main/java/com/pinterest/secor/util/StatsUtil.java new file mode 100644 index 000000000..9cb924dad --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/StatsUtil.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.util; + +import com.twitter.ostrich.stats.Stats; + +/** + * Utilities to interact with Ostrich stats exporter. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class StatsUtil { + public static void setLabel(String name, String value) { + long threadId = Thread.currentThread().getId(); + name += "." + threadId; + Stats.setLabel(name, value); + } + + public static void clearLabel(String name) { + long threadId = Thread.currentThread().getId(); + name += "." + threadId; + Stats.clearLabel(name); + } +} diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java new file mode 100644 index 000000000..64bb236dd --- /dev/null +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.writer; + +import com.pinterest.secor.common.*; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.ParsedMessage; + +import java.io.IOException; + +import com.pinterest.secor.util.IdUtil; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.SequenceFile; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Message writer appends Kafka messages to local log files. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class MessageWriter { + private static final Logger LOG = LoggerFactory.getLogger(MessageWriter.class); + + private SecorConfig mConfig; + private OffsetTracker mOffsetTracker; + private FileRegistry mFileRegistry; + + public MessageWriter(SecorConfig config, OffsetTracker offsetTracker, + FileRegistry fileRegistry) { + mConfig = config; + mOffsetTracker = offsetTracker; + mFileRegistry = fileRegistry; + } + + private void adjustOffset(ParsedMessage message) throws IOException { + TopicPartition topicPartition = new TopicPartition(message.getTopic(), + message.getKafkaPartition()); + long lastSeenOffset = mOffsetTracker.getLastSeenOffset(topicPartition); + if (message.getOffset() != lastSeenOffset + 1) { + // There was a rebalancing event since we read the last message. + LOG.debug("offset of message " + message + + " does not follow sequentially the last seen offset " + lastSeenOffset + + ". Deleting files in topic " + topicPartition.getTopic() + " partition " + + topicPartition.getPartition()); + mFileRegistry.deleteTopicPartition(topicPartition); + } + mOffsetTracker.setLastSeenOffset(topicPartition, message.getOffset()); + } + + public void write(ParsedMessage message) throws IOException { + adjustOffset(message); + TopicPartition topicPartition = new TopicPartition(message.getTopic(), + message.getKafkaPartition()); + long offset = mOffsetTracker.getAdjustedCommittedOffsetCount(topicPartition); + String localPrefix = mConfig.getLocalPath() + '/' + IdUtil.getLocalMessageDir(); + LogFilePath path = new LogFilePath(localPrefix, mConfig.getGeneration(), offset, message); + LongWritable key = new LongWritable(message.getOffset()); + BytesWritable value = new BytesWritable(message.getPayload()); + SequenceFile.Writer writer = mFileRegistry.getOrCreateWriter(path); + writer.append(key, value); + LOG.debug("appended message " + message + " to file " + path.getLogFilePath() + + ". File length " + writer.getLength()); + } +} diff --git a/src/main/scripts/run_consumer.sh b/src/main/scripts/run_consumer.sh new file mode 100755 index 000000000..834cc71f5 --- /dev/null +++ b/src/main/scripts/run_consumer.sh @@ -0,0 +1,30 @@ +#!/bin/sh + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Author: Pawel Garbacki (pawel@pinterest.com) + +mkdir -p /mnt/secor_data/logs + +echo "starting backup group" +nohup java -ea -Dsecor_group=backup -Dlog4j.configuration=log4j.prod.properties \ + -Dconfig=secor.prod.backup.properties -cp "secor-0.1-SNAPSHOT.jar:lib/*" \ + com.pinterest.secor.main.ConsumerMain > /mnt/secor_data/logs/run_consumer_backup.log 2>&1 & + +echo "starting partition group" +nohup java -ea -Dsecor_group=partition -Dlog4j.configuration=log4j.prod.properties \ + -Dconfig=secor.prod.partition.properties -cp "secor-0.1-SNAPSHOT.jar:lib/*" \ + com.pinterest.secor.main.ConsumerMain > /mnt/secor_data/logs/run_secor_partition.log 2>&1 & diff --git a/src/main/scripts/run_kafka_class.sh b/src/main/scripts/run_kafka_class.sh new file mode 100755 index 000000000..86764f5ad --- /dev/null +++ b/src/main/scripts/run_kafka_class.sh @@ -0,0 +1,65 @@ +#!/usr/bin/env bash + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Author: Pawel Garbacki (pawel@pinterest.com) + +set -e + +if [ $# -lt 1 ]; then + echo "USAGE: $0 classname [opts]" + exit 1 +fi + +base_dir=$(dirname $0)/.. + +SCALA_VERSION=2.8.0 + +# assume all dependencies have been packaged into one jar with sbt-assembly's task +# "assembly-package-dependency" +# for file in lib/*.jar; do +# CLASSPATH=$CLASSPATH:$file +# done + +# for file in $base_dir/kafka*.jar; do +# CLASSPATH=$CLASSPATH:$file +# done + +CLASSPATH=${CLASSPATH}:${base_dir}/lib/* + +# JMX settings +KAFKA_JMX_OPTS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false " + +# Log4j settings +KAFKA_LOG4J_OPTS="-Dlog4j.configuration=log4j.dev.properties" + +# Generic jvm settings you want to add +KAFKA_OPTS="" + +# Which java to use +if [ -z "${JAVA_HOME}" ]; then + JAVA="java" +else + JAVA="${JAVA_HOME}/bin/java" +fi + +# Memory options +KAFKA_HEAP_OPTS="-Xmx256M" + +# JVM performance options +KAFKA_JVM_PERFORMANCE_OPTS="-server -XX:+UseCompressedOops -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+CMSClassUnloadingEnabled -XX:+CMSScavengeBeforeRemark -XX:+DisableExplicitGC -Djava.awt.headless=true" + +exec $JAVA $KAFKA_HEAP_OPTS $KAFKA_JVM_PERFORMANCE_OPTS $KAFKA_GC_LOG_OPTS $KAFKA_JMX_OPTS $KAFKA_LOG4J_OPTS -cp $CLASSPATH $KAFKA_OPTS "$@" diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh new file mode 100755 index 000000000..eb3d47dc5 --- /dev/null +++ b/src/main/scripts/run_tests.sh @@ -0,0 +1,217 @@ +#!/usr/bin/env bash + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# End-to-end test exercising the entire stack on the local machine. The test starts a Zookeeper, +# a Kafka broker, and Secor on local host. It then creates a topic and publishes a few messages. +# Secor is given some time to consume them. After that time expires, servers are shut down and +# the output files are verified. +# +# To run the test: +# cd ${OPTIMUS}/secor +# mvn package +# mkdir /tmp/test +# cd /tmp/test +# tar -zxvf ~/git/optimus/secor/target/secor-0.1-SNAPSHOT-bin.tar.gz +# ./scripts/run_tests.sh +# +# Test logs are available in /tmp/secor_dev/logs/ The output files are in +# s3://pinterest-dev/secor_dev + +# Author: Pawel Garbacki (pawel@pinterest.com) + +set -e + +PARENT_DIR=/tmp/secor_dev +LOGS_DIR=${PARENT_DIR}/logs +S3_LOGS_DIR=s3://pinterest-dev/secor_dev +MESSAGES=1000 + +# The minimum wait time is one minute plus delta. Secor is configured to upload files older than +# one minute and we need to make sure that everything ends up on s3 before starting verification. +WAIT_TIME=120 +base_dir=$(dirname $0) + +run_command() { + echo "running $@" + eval "$@" +} + +recreate_dirs() { + run_command "rm -r -f ${PARENT_DIR}" + run_command "s3cmd del --recursive ${S3_LOGS_DIR}" + # create logs directory + if [ ! -d ${LOGS_DIR} ]; then + run_command "mkdir -p ${LOGS_DIR}" + fi +} + +start_zookeeper() { + run_command "${base_dir}/run_kafka_class.sh \ + org.apache.zookeeper.server.quorum.QuorumPeerMain zookeeper.test.properties > \ + ${LOGS_DIR}/zookeeper.log 2>&1 &" +} + +stop_zookeeper() { + run_command "ps ax | grep -i 'org.apache.zookeeper.server.quorum.QuorumPeerMain' | \ + grep -v grep | awk '{print \$1}' | xargs kill" +} + +start_kafka_server () { + run_command "${base_dir}/run_kafka_class.sh kafka.Kafka kafka.test.properties > \ + ${LOGS_DIR}/kafka_server.log 2>&1 &" +} + +stop_kafka_server() { + run_command "ps ax | grep -i 'kafka\.Kafka' | grep java | grep -v grep | awk '{print \$1}' | \ + xargs kill" +} + +start_secor() { + run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ + -Dconfig=secor.dev.backup.properties -cp secor-0.1-SNAPSHOT.jar:lib/* \ + com.pinterest.secor.main.ConsumerMain > ${LOGS_DIR}/secor_backup.log 2>&1 &" + run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ + -Dconfig=secor.dev.partition.properties -cp secor-0.1-SNAPSHOT.jar:lib/* \ + com.pinterest.secor.main.ConsumerMain > ${LOGS_DIR}/secor_partition.log 2>&1 &" +} + +stop_secor() { + run_command "ps ax | grep -i 'com\.pinterest\.secor\.main\.ConsumerMain' | grep java | \ + grep -v grep | awk '{print \$1}' | xargs kill" +} + +create_topic() { + run_command "${base_dir}/run_kafka_class.sh kafka.admin.CreateTopicCommand --zookeeper \ + localhost:2181 --replica 1 --partition 2 --topic test > \ + ${LOGS_DIR}/create_topic.log 2>&1" +} + +post_messages() { + run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ + -Dconfig=secor.dev.backup.properties -cp secor-0.1-SNAPSHOT.jar:lib/* \ + com.pinterest.secor.main.TestLogMessageProducerMain -t test -m $1 -p 1 > \ + ${LOGS_DIR}/test_log_message_producer.log 2>&1" +} + +verify() { + run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ + -Dconfig=secor.dev.backup.properties -cp secor-0.1-SNAPSHOT.jar:lib/* \ + com.pinterest.secor.main.LogFileVerifierMain -t test -m $1 -q > \ + ${LOGS_DIR}/log_verifier_backup.log 2>&1" + run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ + -Dconfig=secor.dev.partition.properties -cp secor-0.1-SNAPSHOT.jar:lib/* \ + com.pinterest.secor.main.LogFileVerifierMain -t test -m $1 -q > \ + ${LOGS_DIR}/log_verifier_partition.log 2>&1" +} + +set_offsets_in_zookeeper() { + for group in secor_backup secor_partition; do + for partition in 0 1; do + run_command "${base_dir}/run_zookeeper_command.sh localhost:2181 create \ + /consumers \'\' > ${LOGS_DIR}/run_zookeeper_command.log 2>&1" + run_command "${base_dir}/run_zookeeper_command.sh localhost:2181 create \ + /consumers/${group} \'\' > ${LOGS_DIR}/run_zookeeper_command.log 2>&1" + run_command "${base_dir}/run_zookeeper_command.sh localhost:2181 create \ + /consumers/${group}/offsets \'\' > ${LOGS_DIR}/run_zookeeper_command.log 2>&1" + run_command "${base_dir}/run_zookeeper_command.sh localhost:2181 create \ + /consumers/${group}/offsets/test \'\' > ${LOGS_DIR}/run_zookeeper_command.log 2>&1" + + run_command "${base_dir}/run_zookeeper_command.sh localhost:2181 create \ + /consumers/${group}/offsets/test/${partition} $1 > \ + ${LOGS_DIR}/run_zookeeper_command.log 2>&1" + done + done +} + +stop_all() { + stop_secor + sleep 1 + stop_kafka_server + sleep 1 + stop_zookeeper + sleep 1 +} + +initialize() { + # Just in case. + stop_all + recreate_dirs + + start_zookeeper + sleep 3 + start_kafka_server + sleep 3 + create_topic + sleep 3 +} + +# Post some messages and verify that they are correctly processes. +post_and_verify_test() { + echo "running post_and_verify_test" + initialize + + start_secor + sleep 3 + post_messages ${MESSAGES} + echo "Waiting ${WAIT_TIME} sec for Secor to upload logs to s3" + sleep ${WAIT_TIME} + verify ${MESSAGES} + + stop_all + echo "post_and_verify_test succeeded" +} + +# Adjust offsets so that Secor consumes only half of the messages. +start_from_non_zero_offset_test() { + echo "running start_from_non_zero_offset_test" + initialize + + set_offsets_in_zookeeper $((${MESSAGES}/4)) + post_messages ${MESSAGES} + start_secor + echo "Waiting ${WAIT_TIME} sec for Secor to upload logs to s3" + sleep ${WAIT_TIME} + verify $((${MESSAGES}/2)) + + stop_all + echo "start_from_non_zero_offset_test succeeded" +} + +# Set offset after consumers processed some of the messages. This scenario simulates a +# rebalancing event and potential topic reassignment triggering the need to trim local log files. +move_offset_back_test() { + echo "running move_offset_back_test" + initialize + + start_secor + sleep 3 + post_messages $((${MESSAGES}/10)) + set_offsets_in_zookeeper 2 + post_messages $((${MESSAGES}*9/10)) + + echo "Waiting ${WAIT_TIME} sec for Secor to upload logs to s3" + sleep ${WAIT_TIME} + # 4 because we skept 2 messages per topic partition and there are 2 partitions per topic. + verify $((${MESSAGES}-4)) + + stop_all + echo "move_offset_back_test succeeded" +} + +post_and_verify_test +start_from_non_zero_offset_test +move_offset_back_test diff --git a/src/main/scripts/run_zookeeper_command.sh b/src/main/scripts/run_zookeeper_command.sh new file mode 100755 index 000000000..8c0128e5b --- /dev/null +++ b/src/main/scripts/run_zookeeper_command.sh @@ -0,0 +1,25 @@ +#!/bin/sh + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Author: Pawel Garbacki (pawel@pinterest.com) + +if [ $# -lt 3 ]; then + echo "USAGE: $0 zookeeper_host:port cmd args" + exit 1 +fi + +java -ea -cp "secor-0.1-SNAPSHOT.jar:lib/*" org.apache.zookeeper.ZooKeeperMain -server $@ diff --git a/src/main/thrift/secor.thrift b/src/main/thrift/secor.thrift new file mode 100644 index 000000000..789f1b5f9 --- /dev/null +++ b/src/main/thrift/secor.thrift @@ -0,0 +1,15 @@ +/* Simple thrift message used in Secor testing */ + +namespace java com.pinterest.secor.thrift + +enum TestEnum { + SOME_VALUE = 0, + SOME_OTHER_VALUE = 1, +} + +struct TestMessage { + 1: required i64 timestamp, + 2: required string requiredField, + 3: optional string optionalField, + 4: optional TestEnum enumField +} diff --git a/src/test/java/com/pinterest/secor/common/FileRegistryTest.java b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java new file mode 100644 index 000000000..4acddb9bf --- /dev/null +++ b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java @@ -0,0 +1,158 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.common; + +import com.pinterest.secor.util.FileUtil; +import junit.framework.TestCase; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.SequenceFile; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.io.IOException; +import java.util.Collection; + +/** + * FileRegistryTest tests the file registry logic. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({FileRegistry.class, FileSystem.class, FileUtil.class, SequenceFile.class}) +public class FileRegistryTest extends TestCase { + private static final String PATH = + "/some_parent_dir/some_topic/some_partition/some_other_partition/" + + "10_0_00000000000000000100"; + private static final String CRC_PATH = + "/some_parent_dir/some_topic/some_partition/some_other_partition/" + + ".10_0_00000000000000000100.crc"; + private LogFilePath mLogFilePath; + private TopicPartition mTopicPartition; + private FileRegistry mRegistry; + + public void setUp() throws Exception { + super.setUp(); + mLogFilePath = new LogFilePath("/some_parent_dir", PATH); + mTopicPartition = new TopicPartition("some_topic", 0); + mRegistry = new FileRegistry(); + } + + private void createWriter() throws IOException { + PowerMockito.mockStatic(FileUtil.class); + + PowerMockito.mockStatic(FileSystem.class); + FileSystem fs = Mockito.mock(FileSystem.class); + Mockito.when(FileSystem.get(Mockito.any(Configuration.class))).thenReturn(fs); + + PowerMockito.mockStatic(SequenceFile.class); + Path fsPath = new Path(PATH); + SequenceFile.Writer writer = Mockito.mock(SequenceFile.Writer.class); + Mockito.when(SequenceFile.createWriter(Mockito.eq(fs), + Mockito.any(Configuration.class), + Mockito.eq(fsPath), + Mockito.eq(LongWritable.class), + Mockito.eq(BytesWritable.class))).thenReturn( + writer); + + Mockito.when(writer.getLength()).thenReturn(123L); + + SequenceFile.Writer createdWriter = mRegistry.getOrCreateWriter(mLogFilePath); + assertTrue(createdWriter == writer); + } + + public void testGetOrCreateWriter() throws Exception { + createWriter(); + + // Call the method again. This time it should return an existing writer. + mRegistry.getOrCreateWriter(mLogFilePath); + + // Verify that the method has been called exactly once (the default). + PowerMockito.verifyStatic(); + FileSystem.get(Mockito.any(Configuration.class)); + + PowerMockito.verifyStatic(); + FileUtil.delete(PATH); + PowerMockito.verifyStatic(); + FileUtil.delete(CRC_PATH); + + Path fsPath = new Path(PATH); + PowerMockito.verifyStatic(); + SequenceFile.createWriter(Mockito.any(FileSystem.class), Mockito.any(Configuration.class), + Mockito.eq(fsPath), Mockito.eq(LongWritable.class), + Mockito.eq(BytesWritable.class)); + + TopicPartition topicPartition = new TopicPartition("some_topic", 0); + Collection topicPartitions = mRegistry.getTopicPartitions(); + assertEquals(1, topicPartitions.size()); + assertTrue(topicPartitions.contains(topicPartition)); + + Collection logFilePaths = mRegistry.getPaths(topicPartition); + assertEquals(1, logFilePaths.size()); + assertTrue(logFilePaths.contains(mLogFilePath)); + } + + public void testDeletePath() throws Exception { + createWriter(); + + PowerMockito.mockStatic(FileUtil.class); + + mRegistry.deletePath(mLogFilePath); + PowerMockito.verifyStatic(); + FileUtil.delete(PATH); + PowerMockito.verifyStatic(); + FileUtil.delete(CRC_PATH); + + assertTrue(mRegistry.getPaths(mTopicPartition).isEmpty()); + assertTrue(mRegistry.getTopicPartitions().isEmpty()); + } + + public void testDeleteTopicPartition() throws Exception { + createWriter(); + + PowerMockito.mockStatic(FileUtil.class); + + mRegistry.deleteTopicPartition(mTopicPartition); + PowerMockito.verifyStatic(); + FileUtil.delete(PATH); + PowerMockito.verifyStatic(); + FileUtil.delete(CRC_PATH); + + assertTrue(mRegistry.getTopicPartitions().isEmpty()); + assertTrue(mRegistry.getPaths(mTopicPartition).isEmpty()); + } + + public void testGetSize() throws Exception { + createWriter(); + + assertEquals(123L, mRegistry.getSize(mTopicPartition)); + } + + public void testGetModificationAgeSec() throws Exception { + PowerMockito.mockStatic(System.class); + PowerMockito.when(System.currentTimeMillis()).thenReturn(10000L).thenReturn(100000L); + createWriter(); + + assertEquals(90, mRegistry.getModificationAgeSec(mTopicPartition)); + } +} diff --git a/src/test/java/com/pinterest/secor/common/LogFilePathTest.java b/src/test/java/com/pinterest/secor/common/LogFilePathTest.java new file mode 100644 index 000000000..dc2baf44e --- /dev/null +++ b/src/test/java/com/pinterest/secor/common/LogFilePathTest.java @@ -0,0 +1,86 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.common; + +import com.pinterest.secor.message.ParsedMessage; +import junit.framework.TestCase; + +import java.util.Arrays; + +/** + * LogFileTest tests the logic operating on lof file paths. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class LogFilePathTest extends TestCase { + private static final String PREFIX = "/some_parent_dir"; + private static final String TOPIC = "some_topic"; + private static final String[] PARTITIONS = {"some_partition", "some_other_partition"}; + private static final int GENERATION = 10; + private static final int KAFKA_PARTITION = 0; + private static final long LAST_COMMITTED_OFFSET = 100; + private static final String PATH = + "/some_parent_dir/some_topic/some_partition/some_other_partition/" + + "10_0_00000000000000000100"; + private static final String CRC_PATH = + "/some_parent_dir/some_topic/some_partition/some_other_partition/" + + ".10_0_00000000000000000100.crc"; + + private LogFilePath mLogFilePath; + + @Override + protected void setUp() throws Exception { + super.setUp(); + mLogFilePath = new LogFilePath(PREFIX, TOPIC, PARTITIONS, GENERATION, KAFKA_PARTITION, + LAST_COMMITTED_OFFSET); + } + + public void testConstructFromMessage() throws Exception { + ParsedMessage message = new ParsedMessage(TOPIC, KAFKA_PARTITION, 1000, + "some_payload".getBytes(), PARTITIONS); + LogFilePath logFilePath = new LogFilePath(PREFIX, GENERATION, LAST_COMMITTED_OFFSET, + message); + assertEquals(PATH, logFilePath.getLogFilePath()); + } + + public void testConstructFromPath() throws Exception { + LogFilePath logFilePath = new LogFilePath("/some_parent_dir", PATH); + + assertEquals(PATH, logFilePath.getLogFilePath()); + assertEquals(TOPIC, logFilePath.getTopic()); + assertTrue(Arrays.equals(PARTITIONS, logFilePath.getPartitions())); + assertEquals(GENERATION, logFilePath.getGeneration()); + assertEquals(KAFKA_PARTITION, logFilePath.getKafkaPartition()); + assertEquals(LAST_COMMITTED_OFFSET, logFilePath.getOffset()); + } + + public void testGetters() throws Exception { + assertEquals(TOPIC, mLogFilePath.getTopic()); + assertTrue(Arrays.equals(PARTITIONS, mLogFilePath.getPartitions())); + assertEquals(GENERATION, mLogFilePath.getGeneration()); + assertEquals(KAFKA_PARTITION, mLogFilePath.getKafkaPartition()); + assertEquals(LAST_COMMITTED_OFFSET, mLogFilePath.getOffset()); + } + + public void testGetLogFilePath() throws Exception { + assertEquals(PATH, mLogFilePath.getLogFilePath()); + } + + public void testGetLogFileCrcPath() throws Exception { + assertEquals(CRC_PATH, mLogFilePath.getLogFileCrcPath()); + } +} diff --git a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java new file mode 100644 index 000000000..d86200552 --- /dev/null +++ b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java @@ -0,0 +1,187 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.uploader; + +import com.pinterest.secor.common.*; +import com.pinterest.secor.util.FileUtil; +import com.pinterest.secor.util.IdUtil; +import junit.framework.TestCase; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.*; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.io.IOException; +import java.util.HashSet; + +/** + * UploaderTest tests the log file uploader logic. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({FileUtil.class, FileSystem.class, IdUtil.class}) +public class UploaderTest extends TestCase { + private static class TestUploader extends Uploader { + private SequenceFile.Reader mReader; + + public TestUploader(SecorConfig config, OffsetTracker offsetTracker, + FileRegistry fileRegistry, ZookeeperConnector zookeeperConnector) { + super(config, offsetTracker, fileRegistry, zookeeperConnector); + mReader = Mockito.mock(SequenceFile.Reader.class); + } + + @Override + protected SequenceFile.Reader createReader(FileSystem fileSystem, Path path, + Configuration configuration) throws IOException { + return mReader; + } + + public SequenceFile.Reader getReader() { + return mReader; + } + } + + private TopicPartition mTopicPartition; + + private LogFilePath mLogFilePath; + + private SecorConfig mConfig; + private OffsetTracker mOffsetTracker; + private FileRegistry mFileRegistry; + private ZookeeperConnector mZookeeperConnector; + + private TestUploader mUploader; + + @Override + public void setUp() throws Exception { + super.setUp(); + mTopicPartition = new TopicPartition("some_topic", 0); + + mLogFilePath = new LogFilePath("/some_parent_dir", + "/some_parent_dir/some_topic/some_partition/some_other_partition/" + + "10_0_00000000000000000010"); + + mConfig = Mockito.mock(SecorConfig.class); + Mockito.when(mConfig.getLocalPath()).thenReturn("/some_parent_dir"); + Mockito.when(mConfig.getMaxFileSizeBytes()).thenReturn(10L); + + mOffsetTracker = Mockito.mock(OffsetTracker.class); + + mFileRegistry = Mockito.mock(FileRegistry.class); + Mockito.when(mFileRegistry.getSize(mTopicPartition)).thenReturn(100L); + HashSet topicPartitions = new HashSet(); + topicPartitions.add(mTopicPartition); + Mockito.when(mFileRegistry.getTopicPartitions()).thenReturn(topicPartitions); + + mZookeeperConnector = Mockito.mock(ZookeeperConnector.class); + mUploader = new TestUploader(mConfig, mOffsetTracker, mFileRegistry, mZookeeperConnector); + } + + public void testUploadFiles() throws Exception { + Mockito.when(mZookeeperConnector.getCommittedOffsetCount(mTopicPartition)).thenReturn(11L); + Mockito.when(mOffsetTracker.setCommittedOffsetCount(mTopicPartition, 11L)).thenReturn(11L); + Mockito.when(mOffsetTracker.setCommittedOffsetCount(mTopicPartition, 21L)).thenReturn(11L); + Mockito.when(mOffsetTracker.getLastSeenOffset(mTopicPartition)).thenReturn(20L); + Mockito.when(mOffsetTracker.getTrueCommittedOffsetCount(mTopicPartition)).thenReturn(11L); + Mockito.when(mConfig.getS3Bucket()).thenReturn("some_bucket"); + Mockito.when(mConfig.getS3Path()).thenReturn("some_s3_parent_dir"); + + HashSet logFilePaths = new HashSet(); + logFilePaths.add(mLogFilePath); + Mockito.when(mFileRegistry.getPaths(mTopicPartition)).thenReturn(logFilePaths); + + PowerMockito.mockStatic(FileUtil.class); + + mUploader.applyPolicy(); + + final String lockPath = "/secor/locks/some_topic/0"; + Mockito.verify(mZookeeperConnector).lock(lockPath); + PowerMockito.verifyStatic(); + FileUtil.moveToS3( + "/some_parent_dir/some_topic/some_partition/some_other_partition/" + + "10_0_00000000000000000010", + "s3n://some_bucket/some_s3_parent_dir/some_topic/some_partition/" + + "some_other_partition/10_0_00000000000000000010"); + Mockito.verify(mFileRegistry).deleteTopicPartition(mTopicPartition); + Mockito.verify(mZookeeperConnector).setCommittedOffsetCount(mTopicPartition, 21L); + Mockito.verify(mOffsetTracker).setCommittedOffsetCount(mTopicPartition, 21L); + Mockito.verify(mZookeeperConnector).unlock(lockPath); + } + + public void testDeleteTopicPartition() throws Exception { + Mockito.when(mZookeeperConnector.getCommittedOffsetCount(mTopicPartition)).thenReturn(31L); + Mockito.when(mOffsetTracker.setCommittedOffsetCount(mTopicPartition, 30L)).thenReturn(11L); + Mockito.when(mOffsetTracker.getLastSeenOffset(mTopicPartition)).thenReturn(20L); + + mUploader.applyPolicy(); + + Mockito.verify(mFileRegistry).deleteTopicPartition(mTopicPartition); + } + + public void testTrimFiles() throws Exception { + Mockito.when(mZookeeperConnector.getCommittedOffsetCount(mTopicPartition)).thenReturn(21L); + Mockito.when(mOffsetTracker.setCommittedOffsetCount(mTopicPartition, 21L)).thenReturn(20L); + Mockito.when(mOffsetTracker.getLastSeenOffset(mTopicPartition)).thenReturn(21L); + + HashSet logFilePaths = new HashSet(); + logFilePaths.add(mLogFilePath); + Mockito.when(mFileRegistry.getPaths(mTopicPartition)).thenReturn(logFilePaths); + + PowerMockito.mockStatic(FileSystem.class); + + SequenceFile.Reader reader = mUploader.getReader(); + Mockito.doReturn(LongWritable.class).when(reader).getKeyClass(); + Mockito.doReturn(BytesWritable.class).when(reader).getValueClass(); + + Mockito.when(reader.next(Mockito.any(Writable.class), + Mockito.any(Writable.class))).thenAnswer(new Answer() { + private int mCallCount = 0; + @Override + public Boolean answer(InvocationOnMock invocation) throws Throwable { + if (mCallCount == 2) { + return false; + } + LongWritable key = (LongWritable) invocation.getArguments()[0]; + key.set(20 + mCallCount++); + return true; + } + }); + + PowerMockito.mockStatic(IdUtil.class); + Mockito.when(IdUtil.getLocalMessageDir()).thenReturn("some_message_dir"); + + SequenceFile.Writer writer = Mockito.mock(SequenceFile.Writer.class); + LogFilePath dstLogFilePath = new LogFilePath("/some_parent_dir/some_message_dir", + "/some_parent_dir/some_message_dir/some_topic/some_partition/" + + "some_other_partition/10_0_00000000000000000021"); + Mockito.when(mFileRegistry.getOrCreateWriter(dstLogFilePath)).thenReturn(writer); + + mUploader.applyPolicy(); + + Mockito.verify(writer).append(Mockito.any(LongWritable.class), + Mockito.any(BytesWritable.class)); + Mockito.verify(mFileRegistry).deletePath(mLogFilePath); + } +}