diff --git a/.gitignore b/.gitignore new file mode 100644 index 000000000..290c183aa --- /dev/null +++ b/.gitignore @@ -0,0 +1,46 @@ +# Compiled source # +################### +*.com +*.class +*.dll +*.exe +*.o +*.so + +# Packages # +############ +# it's better to unpack these files and commit the raw source +# git has its own built in compression methods +*.7z +*.dmg +*.gz +*.iso +*.jar +*.rar +*.tar +*.zip + +# Logs and databases # +###################### +*.log +*.sql +*.sqlite + +# OS generated files # +###################### +.DS_Store +.DS_Store? +._* +.Spotlight-V100 +.Trashes +ehthumbs.db +Thumbs.db + +*.iml +.idea +target +bin/ +.settings/ +.project +.classpath +.m2 diff --git a/.travis.yml b/.travis.yml new file mode 100644 index 000000000..1f45b397e --- /dev/null +++ b/.travis.yml @@ -0,0 +1,19 @@ +language: java +addons: + hosts: + - test-bucket.localhost +env: + - PATH=$PATH:$HOME/.s3cmd SECOR_LOCAL_S3=true S3CMD=1.0.1 +jdk: + - openjdk7 + - oraclejdk8 +before_install: + - wget https://github.com/s3tools/s3cmd/archive/v$S3CMD.tar.gz -O /tmp/s3cmd.tar.gz + - tar -xzf /tmp/s3cmd.tar.gz -C $HOME + - mv $HOME/s3cmd-$S3CMD $HOME/.s3cmd + - cd $HOME/.s3cmd && python setup.py install --user && cd - + - gem install fakes3 -v 0.1.7 +script: + - make unit + - make integration + diff --git a/DESIGN.md b/DESIGN.md index c41ed1c47..24c82ad58 100644 --- a/DESIGN.md +++ b/DESIGN.md @@ -24,14 +24,12 @@ This document assumes familiarity with [Apache Kafka]. * **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). +* **dependence on public APIs:** the system should rely on public [Kafka] APIs only. Furthermore, it should be compatible with [Kafka] version 0.8. 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 @@ -160,10 +158,47 @@ uploader.check_policy() { 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 `` <= ``. +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. +## Output file formats + +Secor supports two different output file formats with different capabilities. + +### Text File + +The Delimited Text File output format writes individual messages as raw bytes, separated by newline characters. Thus, +it is generally only appropriate for non binary messages that do not contain embedded newlines. No other metadata +about the message is recorded. + +### Hadoop SequenceFile + +The [SequenceFile](https://wiki.apache.org/hadoop/SequenceFile) format writes out the message body in the **value** +field of the SequenceFile record. It supports two different modes for storing additional metadata in the **key** +field of the SequenceFile. + +#### Legacy + +In the default, legacy mode, the kafka partition offset is stored in the key field as an 8 byte long value in big +endian format. + +#### MessagePack + +In the optional, [MessagePack](http://msgpack.org/index.html) mode, the key is a binary structure encoded using the +MessagePack specification. MessagePack is a hierarchical map datastructure like JSON, but has a more compact, binary +representation, and support for more types. + +The MessagePack map stored in the SequenceFile key has its Secor keys stored using integer values, for compactness. +The currently defined Secor keys, their meanings, and their associated MessagePack value types are explained below. + +| Key | Meaning | MessagePack Value Type | +| ------------ | ---------------------- | ---------------------- | +| 1 | kafka partition offset | 64 bit Integer | +| 2 | kafka message key | Raw Binary byte array | + +Note that if the kafka message has no key, then the field will be omitted from the the MessagePack. + ## 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. diff --git a/Dockerfile b/Dockerfile new file mode 100644 index 000000000..6504267b0 --- /dev/null +++ b/Dockerfile @@ -0,0 +1,9 @@ +FROM openjdk:8 + +RUN mkdir -p /opt/secor +ADD target/secor-*-bin.tar.gz /opt/secor/ + +COPY src/main/scripts/docker-entrypoint.sh /docker-entrypoint.sh +RUN chmod +x /docker-entrypoint.sh + +ENTRYPOINT ["/docker-entrypoint.sh"] diff --git a/Makefile b/Makefile new file mode 100644 index 000000000..73de33174 --- /dev/null +++ b/Makefile @@ -0,0 +1,35 @@ +CONFIG=src/main/config +TEST_HOME=/tmp/secor_test +TEST_CONFIG=src/test/config +JAR_FILE=target/secor-*-SNAPSHOT-bin.tar.gz +MVN_OPTS=-DskipTests=true -Dmaven.javadoc.skip=true +CONTAINERS=$(shell ls containers) + +build: + @mvn package $(MVN_OPTS) + +unit: + @mvn test + +integration: build + @rm -rf $(TEST_HOME) + @mkdir -p $(TEST_HOME) + @tar -xzf $(JAR_FILE) -C $(TEST_HOME) + @cp $(TEST_CONFIG)/* $(TEST_HOME) + @[ ! -e $(CONFIG)/core-site.xml ] && jar uf $(TEST_HOME)/secor-*.jar -C $(TEST_CONFIG) core-site.xml + @[ ! -e $(CONFIG)/jets3t.properties ] && jar uf $(TEST_HOME)/secor-*.jar -C $(TEST_CONFIG) jets3t.properties + cd $(TEST_HOME) && ./scripts/run_tests.sh + +test: build unit integration + +container_%: + docker build -t secor_$* containers/$* + +test_%: container_% + @mkdir -p .m2 + docker run -v $(CURDIR)/.m2:/root/.m2:rw -v $(CURDIR):/work:rw secor_$* sh -c "echo 127.0.0.1 test-bucket.localhost >> /etc/hosts && make clean test" + +docker_test: $(foreach container, $(CONTAINERS), test_$(container)) + +clean: + rm -rf target/ diff --git a/README.md b/README.md index 79c46d15b..6886f6432 100644 --- a/README.md +++ b/README.md @@ -1,16 +1,19 @@ # Pinterest Secor -Secor is a service persisting [Kafka] logs to [Amazon S3]. +[![Build Status](https://travis-ci.org/pinterest/secor.svg)](https://travis-ci.org/pinterest/secor) -## 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, +Secor is a service persisting [Kafka] logs to [Amazon S3], [Google Cloud Storage], [Microsoft Azure Blob Storage] and [Openstack Swift]. + +## Key features ## + - **strong consistency**: as long as [Kafka] is not dropping messages (e.g., due to aggressive cleanup policy) before Secor is able to read them, it is guaranteed that each message will be saved in exactly one [S3] file. This property is not compromised by the notorious temporal inconsistency 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], + - **output partitioning**: Secor parses incoming messages and puts them under partitioned s3 paths to enable direct import into systems like [Hive]. day,hour,minute level partitions are supported by secor - **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], + - **monitoring**: metrics tracking various performance properties are exposed through [Ostrich] and optionally exported to [OpenTSDB] / [statsD], - **customizability**: external log message parser may be loaded by updating the configuration, + - **event transformation**: external message level transformation can be done by using customized class. - **Qubole interface**: Secor connects to [Qubole] to add finalized output partitions to Hive tables. ## Setup Guide @@ -26,9 +29,13 @@ Edit `src/main/config/*.properties` files to specify parameters describing the e ##### Create and install jars ```sh +# By default this will install the "release" (Kafka 0.10 profile) 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} + +# To use the Kafka 0.8 client you should use the kafka-0.8-dev profile +mvn -Pkafka-0.8-dev package ``` ##### Run tests (optional) @@ -40,7 +47,11 @@ cd ${SECOR_INSTALL_DIR} ##### Run Secor ```sh cd ${SECOR_INSTALL_DIR} -java -ea -Dsecor_group=secor_backup -Dlog4j.configuration=log4j.prod.properties -Dconfig=secor.prod.backup.properties -cp secor-0.1-SNAPSHOT.jar:lib/* com.pinterest.secor.main.ConsumerMain +java -ea -Dsecor_group=secor_backup \ + -Dlog4j.configuration=log4j.prod.properties \ + -Dconfig=secor.prod.backup.properties \ + -cp secor-0.1-SNAPSHOT.jar:lib/* \ + com.pinterest.secor.main.ConsumerMain ``` ## Output grouping @@ -49,14 +60,49 @@ One of the convenience features of Secor is the ability to group messages and sa - **offset parser**: parser that groups messages based on offset ranges. E.g., messages with offsets in range 0 to 999 will end up under ```s3n://bucket/topic/offset=0/```, offsets 1000 to 2000 will go to ```s3n://bucket/topic/offset=1000/```. To use this parser, start Secor with properties file [secor.prod.backup.properties](src/main/config/secor.prod.backup.properties). -- **thrift date parser**: parser that extracts timestamps from thrift messages and groups the output based on the date (at a day granularity). To keep things simple, this parser assumes that the timestamp is carried in the first field (id 0) of the thrift message schema. The timestamp may be expressed either in seconds or milliseconds, or nanoseconds since the epoch. The output goes to date-partitioned paths (e.g., ```s3n://bucket/topic/dt=2014-05-01```, ```s3n://bucket/topic/dt=2014-05-02```). Date pertitioning is particularly convenient if the output is to be consumed by ETL tools such as [Hive]. To use this parser, start Secor with properties file [secor.prod.partition.properties](src/main/config/secor.prod.partition.properties). You may override the field used to extract the timestamp by setting the "message.timestamp.name" property. +- **Thrift date parser**: parser that extracts timestamps from thrift messages and groups the output based on the date (at a day granularity). To keep things simple, this parser assumes that the timestamp is carried in the first field (id 1) of the thrift message schema by default. The field id can be changed by setting ```message.timestamp.id``` as long as the field is at the top level of the thrift object (i.e. it is not in a nested structure). The timestamp may be expressed either in seconds or milliseconds, or nanoseconds since the epoch. The output goes to date-partitioned paths (e.g., ```s3n://bucket/topic/dt=2014-05-01```, ```s3n://bucket/topic/dt=2014-05-02```). Date partitioning is particularly convenient if the output is to be consumed by ETL tools such as [Hive]. To use this parser, start Secor with properties file [secor.prod.partition.properties](src/main/config/secor.prod.partition.properties). Note the ```message.timestamp.name``` property has no effect on the thrift parsing, which is determined by the field id. + +- **JSON timestamp parser**: parser that extracts UNIX timestamps from JSON messages and groups the output based on the date, similar to the Thrift parser above. To use this parser, start Secor with properties file [secor.prod.partition.properties](src/main/config/secor.prod.partition.properties) and set `secor.message.parser.class=com.pinterest.secor.parser.JsonMessageParser`. You may override the field used to extract the timestamp by setting the "message.timestamp.name" property. + +- **JSON ISO 8601 date parser**: Assumes your timestamp field uses ISO 8601. To use this parser, start Secor with properties file [secor.prod.partition.properties](src/main/config/secor.prod.partition.properties) and set `secor.message.parser.class=com.pinterest.secor.parser.Iso8601MessageParser`. You may override the field used to extract the timestamp by setting the "message.timestamp.name" property. + +- **MessagePack date parser**: parser that extracts timestamps from MessagePack messages and groups the output based on the date, similar to the Thrift and JSON parser. To use this parser, set `secor.message.parser.class=com.pinterest.secor.parser.MessagePackParser`. Like the Thrift parser, the timestamp may be expressed either in seconds or milliseconds, or nanoseconds since the epoch and respects the "message.timestamp.name" property. + +- **[Protocol Buffers]** date parser: parser that extracts timestamps from protobuf messages and groups the output based on the date, similar to the Thrift, JSON or MessagePack parser. To use this parser, set `secor.message.parser.class=com.pinterest.secor.parser.ProtobufMessageParser`. Like the Thrift parser, the timestamp may be expressed either in seconds or milliseconds, or nanoseconds since the epoch and respects the "message.timestamp.name" property. -- **JSON date parser**: parser that extracts timestamps from JSON messages and groups the output based on the date, similar to the Thrift parser above. To use this parser, start Secor with properties file [secor.prod.partition.properties](src/main/config/secor.prod.partition.properties) and set `secor.message.parser.class=com.pinterest.secor.parser.JsonMessageParser`. You may override the field used to extract the timestamp by setting the "message.timestamp.name" property. +- **Output grouping with Flexible partitions**: The default partitioning granularity for date, hours and minutes have prefix for convenient consumption for `Hive`. If you require different naming of partition with(out) prefix and other date, hour or minute format update the following properties in `secor.common.properties` + + partitioner.granularity.date.prefix=dt= + partitioner.granularity.hour.prefix=hr= + partitioner.granularity.minute.prefix=min= + + partitioner.granularity.date.format=yyyy-MM-dd + partitioner.granularity.hour.format=HH + partitioner.granularity.minute.format=mm + If none of the parsers available out-of-the-box is suitable for your use case, note that it is very easy to implement a custom parser. All you have to do is to extend [MessageParser](src/main/java/com/pinterest/secor/parser/MessageParser.java) and tell Secor to use your parser by setting ```secor.message.parser.class``` in the properties file. + +## Output File Formats + +Currently secor supports the following output formats + +- **Sequence Files**: Flat file containing binary key value pairs. To use this format, set `secor.file.reader.writer.factory=com.pinterest.secor.io.impl.SequenceFileReaderWriterFactory` option. + +- **Delimited Text Files**: A new line delimited raw text file. To use this format, set `secor.file.reader.writer.factory=com.pinterest.secor.io.impl.DelimitedTextFileReaderWriterFactory` option. + +- **ORC Files**: Optimized row columnar format. To use this format, set `secor.file.reader.writer.factory=com.pinterest.secor.io.impl.JsonORCFileReaderWriterFactory` option. Additionally, ORC schema must be specified per topic like this `secor.orc.message.schema.=`. If all Kafka topics receive same format data then this option can be used `secor.orc.message.schema.*=`. User can implement custom ORC schema provider by implementing ORCScehmaProvider interface and the new provider class should be specified using option `secor.orc.schema.provider=`. By default this property is DefaultORCSchemaProvider. + +- **[Parquet] Files (for Protobuf messages)**: Columnar storage format. To use this output format, set `secor.file.reader.writer.factory=com.pinterest.secor.io.impl.ProtobufParquetFileReaderWriterFactory` option. In addition, Protobuf message class per Kafka topic must be defined using option `secor.protobuf.message.class.=`. If all Kafka topics transfer the same protobuf message type, set `secor.protobuf.message.class.*=`. + +- **[Parquet] Files (for Thrift messages)**: Columnar storage format. To use this output format, set `secor.file.reader.writer.factory=com.pinterest.secor.io.impl.ThriftParquetFileReaderWriterFactory` option. In addition, thrift message class per Kafka topic must be defined using option `secor.thrift.message.class.=`. If all Kafka topics transfer the same thrift message type, set `secor.thrift.message.class.*=`. It is asumed all messages use the same thrift protocol. Thrift protocol is set in `secor.thrift.protocol.class`. + +- **Gzip upload format**: To enable compression on uploaded files to the cloud, in `secor.common.properties` set `secor.compression.codec` to a valid compression codec implementing `org.apache.hadoop.io.compress.CompressionCodec` interface, such as `org.apache.hadoop.io.compress.GzipCodec`. + + ## Tools -Secor comes with a number of tools impelementing interactions with the environment. +Secor comes with a number of tools implementing interactions with the environment. ##### Log file printer Log file printer displays the content of a log file. @@ -73,19 +119,22 @@ java -ea -Dlog4j.configuration=log4j.prod.properties -Dconfig=secor.prod.backup. ``` ##### 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. +Topic finalizer writes _SUCCESS files to date partitions that very likely won't be receiving any new messages and (optionally) 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 +java -ea -Dlog4j.configuration=log4j.prod.properties -Dconfig=secor.prod.backup.properties -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. +Progress monitor exports offset consumption lags per topic partition to [OpenTSDB] / [statsD]. 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 ``` +Set `monitoring.interval.seconds` to a value larger than 0 to run in a loop, exporting stats every `monitoring.interval.seconds` seconds. + + ## Detailed design Design details are available in [DESIGN.md](DESIGN.md). @@ -96,22 +145,56 @@ Secor is distributed under [Apache License, Version 2.0](http://www.apache.org/l ## Maintainers * [Pawel Garbacki](https://github.com/pgarbacki) + * [Henry Cai](https://github.com/HenryCaiHaiying) ## Contributors * [Andy Kramolisch](https://github.com/andykram) * [Brenden Matthews](https://github.com/brndnmtthws) * [Lucas Zago](https://github.com/zago) + * [James Green](https://github.com/jfgreen) + * [Praveen Murugesan](https://github.com/lefthandmagic) + * [Zack Dever](https://github.com/zackdever) + * [Leo Woessner](https://github.com/estezz) + * [Jerome Gagnon](https://github.com/jgagnon1) + * [Taichi Nakashima](https://github.com/tcnksm) + * [Lovenish Goyal](https://github.com/lovenishgoyal) + * [Ahsan Nabi Dar](https://github.com/ahsandar) + * [Ashish Kumar](https://github.com/ashubhumca) + * [Ashwin Sinha](https://github.com/tygrash) + + +## Companies who use Secor + + * [Airbnb](https://www.airbnb.com) + * [Pinterest](https://www.pinterest.com) + * [Strava](https://www.strava.com) + * [TiVo](https://www.tivo.com) + * [Yelp](http://www.yelp.com) + * [Credit Karma](https://www.creditkarma.com) + * [VarageSale](http://www.varagesale.com) + * [Skyscanner](http://www.skyscanner.net) + * [Nextperf](http://www.nextperf.com) + * [Zalando](http://www.zalando.com) + * [Rakuten](http://techblog.rakuten.co.jp/) + * [Appsflyer](https://www.appsflyer.com) + * [Wego](https://www.wego.com) + * [GO-JEK](http://gojekengineering.com/) ## Help -If you have any questions or comments, you can reach us at [secor-users@googlegroups.com](secor-users@googlegroups.com) +If you have any questions or comments, you can reach us at [secor-users@googlegroups.com](https://groups.google.com/forum/#!forum/secor-users) [Kafka]:http://kafka.apache.org/ [Amazon S3]:http://aws.amazon.com/s3/ +[Microsoft Azure Blob Storage]:https://azure.microsoft.com/en-us/services/storage/blobs/ [S3]:http://aws.amazon.com/s3/ +[Google Cloud Storage]:https://cloud.google.com/storage/ [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/ - +[statsD]: https://github.com/etsy/statsd/ +[Openstack Swift]: http://swift.openstack.org +[Protocol Buffers]: https://developers.google.com/protocol-buffers/ +[Parquet]: https://parquet.apache.org/ diff --git a/containers/trusty/Dockerfile b/containers/trusty/Dockerfile new file mode 100644 index 000000000..670bdb870 --- /dev/null +++ b/containers/trusty/Dockerfile @@ -0,0 +1,9 @@ +FROM ubuntu:trusty + +RUN apt-get update && \ + apt-get -y install git make maven openjdk-7-jdk ruby s3cmd wget && \ + gem install fakes3 -v 0.1.7 + +ENV SECOR_LOCAL_S3 true + +WORKDIR /work diff --git a/containers/xenial/Dockerfile b/containers/xenial/Dockerfile new file mode 100644 index 000000000..41ed89d8d --- /dev/null +++ b/containers/xenial/Dockerfile @@ -0,0 +1,9 @@ +FROM ubuntu:xenial + +RUN apt-get update && \ + apt-get -y install git make maven openjdk-8-jdk-headless ruby s3cmd wget && \ + gem install fakes3 -v 0.2.4 + +ENV SECOR_LOCAL_S3 true + +WORKDIR /work diff --git a/pom.xml b/pom.xml index b08a4d752..67eee6df5 100644 --- a/pom.xml +++ b/pom.xml @@ -4,11 +4,13 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - com.pinterest.secor + com.pinterest secor - 0.1-SNAPSHOT + 0.26-SNAPSHOT jar - Kafka to s3 logs exporter + secor + Kafka to s3/gs/swift logs exporter + https://github.com/pinterest/secor @@ -19,13 +21,50 @@ + + + pgarbacki + Pawel Garbacki + + + yuyang + Yu Yang + + + ramki + Ramki Venkatachalam + + + hcai + Henry Cai + + + + + https://github.com/pinterest/secor.git + https://github.com/pinterest/secor.git + https://github.com/pinterest/secor + + 1.6 1.6 UTF-8 UTF-8 + 1.9.0 + + + ossrh + https://oss.sonatype.org/service/local/staging/deploy/maven2/ + + + ossrh + https://oss.sonatype.org/content/repositories/snapshots + + + Twitter public Maven repo @@ -49,20 +88,34 @@ - org.apache.kafka - kafka_2.10 - 0.8.1.1 - - - org.slf4j - slf4j-simple - - + com.google.protobuf + protobuf-java + 3.1.0 + + + com.google.protobuf + protobuf-java-util + 3.1.0 + + + com.amazonaws + aws-java-sdk-s3 + 1.11.160 + + + com.amazonaws + aws-java-sdk-sts + 1.11.160 + + + net.java.dev.jets3t + jets3t + 0.9.4 log4j log4j - 1.2.15 + 1.2.17 com.sun.jmx @@ -98,10 +151,80 @@ commons-configuration 1.9 + + + org.apache.hadoop + hadoop-common + 2.7.0 + + + org.apache.hadoop + hadoop-mapreduce-client-core + 2.7.0 + org.apache.hadoop - hadoop-core - 1.2.1 + hadoop-aws + 2.7.0 + + + net.java.dev.jets3t + jets3t + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.amazonaws + aws-java-sdk + + + + + org.apache.hadoop + hadoop-openstack + 2.7.0 + + + + + org.apache.parquet + parquet-common + ${parquet.version} + + + org.apache.parquet + parquet-encoding + ${parquet.version} + + + org.apache.parquet + parquet-column + ${parquet.version} + + + org.apache.parquet + parquet-hadoop + ${parquet.version} + + + org.apache.parquet + parquet-protobuf + ${parquet.version} + + + org.apache.parquet + parquet-thrift + ${parquet.version} org.apache.thrift @@ -121,7 +244,7 @@ com.google.guava guava - 14.0 + 18.0 net.minidev @@ -151,6 +274,47 @@ 1.5.2 test + + org.msgpack + jackson-dataformat-msgpack + 0.7.0-p2 + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + + + com.timgroup + java-statsd-client + 3.0.2 + + + com.google.apis + google-api-services-storage + v1-rev40-1.20.0 + + + com.google.guava + guava-jdk5 + + + + + com.microsoft.azure + azure-storage + 4.0.0 + + + org.apache.orc + orc-core + 1.3.3 + @@ -159,6 +323,11 @@ src/main/config + + + src/test/config + + org.codehaus.mojo @@ -236,7 +405,58 @@ - + + + + com.github.os72 + protoc-jar-maven-plugin + 3.1.0 + + + protobuf-test-sources + generate-test-sources + + run + + + 3.1.0 + + src/test/protobuf + + + src/test/protobuf + + true + + + java + none + target/generated-test-sources/protobuf/gen-java + + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + 1.10 + + + add-test-sources + generate-test-sources + + add-test-source + + + + target/generated-test-sources/protobuf/gen-java + + + + + com.twitter @@ -293,6 +513,149 @@ + + org.apache.maven.plugins + maven-jar-plugin + 2.4 + + + + test-jar + + + + + + org.sonatype.plugins + nexus-staging-maven-plugin + 1.6.3 + true + + ossrh + https://oss.sonatype.org/ + true + + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.7 + 1.7 + + + + + + release + + + + org.apache.maven.plugins + maven-source-plugin + 2.2.1 + + + attach-sources + + jar-no-fork + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + 2.9.1 + + + attach-javadocs + + jar + + + + + + org.apache.maven.plugins + maven-gpg-plugin + 1.5 + + + sign-artifacts + verify + + sign + + + + + + + + + org.apache.kafka + kafka_2.10 + 0.10.2.0 + + + org.slf4j + slf4j-simple + + + + + + + kafka-0.8-dev + + + + org.apache.maven.plugins + maven-compiler-plugin + 2.0.2 + + + com/pinterest/secor/timestamp/Kafka10MessageTimestamp.java + + + + + + + + org.apache.kafka + kafka_2.10 + 0.8.2.1 + + + org.slf4j + slf4j-simple + + + + + + + kafka-0.10-dev + + true + + + + org.apache.kafka + kafka_2.10 + 0.10.2.0 + + + org.slf4j + slf4j-simple + + + + + + diff --git a/src/main/config/kafka.test.properties b/src/main/config/kafka.test.properties index e79bbc8a5..5a41aa2cf 100644 --- a/src/main/config/kafka.test.properties +++ b/src/main/config/kafka.test.properties @@ -47,7 +47,7 @@ socket.request.max.bytes=104857600 ############################# Log Basics ############################# -# A comma seperated list of directories under which to store log files +# A comma separated 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 @@ -61,7 +61,7 @@ num.partitions=2 # 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. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to excessive 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. diff --git a/src/main/config/log4j.dev.properties b/src/main/config/log4j.dev.properties index 720a7907d..a0b78828a 100644 --- a/src/main/config/log4j.dev.properties +++ b/src/main/config/log4j.dev.properties @@ -5,6 +5,7 @@ log4j.rootLogger=DEBUG, CONSOLE, ROLLINGFILE log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender log4j.appender.CONSOLE.Threshold=INFO +log4j.appender.CONSOLE.Target=System.err log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} [%t] (%C:%L) %-5p %m%n diff --git a/src/main/config/log4j.docker.properties b/src/main/config/log4j.docker.properties new file mode 100644 index 000000000..06a117682 --- /dev/null +++ b/src/main/config/log4j.docker.properties @@ -0,0 +1,8 @@ +# log4j logging configuration. + +# root logger. +log4j.rootLogger=INFO, CONSOLE + +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} [%t] (%c) %-5p %m%n diff --git a/src/main/config/log4j.prod.properties b/src/main/config/log4j.prod.properties index 298e2cdb9..fce4ca6c7 100644 --- a/src/main/config/log4j.prod.properties +++ b/src/main/config/log4j.prod.properties @@ -5,6 +5,7 @@ log4j.rootLogger=DEBUG, CONSOLE, ROLLINGFILE log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender log4j.appender.CONSOLE.Threshold=WARN +log4j.appender.CONSOLE.Target=System.err log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout log4j.appender.CONSOLE.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 index 80601c105..5810d979d 100644 --- a/src/main/config/secor.common.properties +++ b/src/main/config/secor.common.properties @@ -19,31 +19,153 @@ # Regular expression matching names of consumed topics. secor.kafka.topic_filter=.* +secor.kafka.topic_blacklist= +# Adding files on S3 path with custom topic name +# Example: secor.kafka.customtopicname.topicname1=customtopicname1 + +# Choose what to fill according to the service you are using +# in the choice option you can fill S3, GS, Swift or Azure +cloud.service=S3 # AWS authentication credentials. +# Leave empty if using IAM role-based authentication with s3a filesystem. aws.access.key= aws.secret.key= +aws.role= + +# Optional Proxy Setting. Set to true to enable proxy +# Only applicable to S3UploadManager +aws.proxy.isEnabled=false +aws.proxy.http.host= +aws.proxy.http.port= ################ # END MUST SET # ################ + +# AWS region or endpoint. region should be a known region name (eg. +# us-east-1). endpoint should be a known S3 endpoint url. If neither +# are specified, then the default region (us-east-1) is used. If both +# are specified then endpoint is used. +# +# Only apply if the the S3UploadManager is used - see +# secor.upload.manager.class. +# +# http://docs.aws.amazon.com/general/latest/gr/rande.html#s3_region +aws.region= +aws.endpoint= + +# Toggle the AWS S3 client between virtual host style access and path style +# access. See http://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html +aws.client.pathstyleaccess=false + +########################### +# START AWS S3 ENCRYPTION # +########################### + +# AWS specify type of server-side encryption, if any +# set to S3 to enable S3-managed encryption +# set to KMS to enable AWS KMS-managed encryption (see aws.sse.kms.key) +# set to customer to enable customer-managed encryption (see aws.sse.customer.key) +# set empty to disable encryption +aws.sse.type= + +# Key to use for S3 server-side encryption, base64-encoded +# Note: requires aws.sse.type to be set to customer to be used +aws.sse.customer.key= + +# KMS Key to use for S3 server-side encryption, base64-encoded +# Leave empty to use default generated key +# Note: requires aws.sse.type to be set to KMS to be used +aws.sse.kms.key= + +######################### +# END AWS S3 ENCRYPTION # +######################### + +# Hadoop filesystem to use. Choices are s3n or s3a. +# See https://wiki.apache.org/hadoop/AmazonS3 for details. +secor.s3.filesystem=s3n + +# Swift config, MUST configure if cloud.service=Swift + +# Swift Login Details: +swift.use.get.auth=true +swift.auth.url= +swift.tenant= +swift.username= +swift.port=8080 +swift.public=true + +# only needed if "swift.use.get.auth" = false +swift.password= + +# only needed if "swift.use.get.auth" = true +swift.api.key= + +# GS config, MUST configure if gcloud.service=GS + +# Name of the Google cloud storage bucket where log files are stored. +secor.gs.bucket=secor_gs + +# Google cloud storage path where files are stored within the bucket. +secor.gs.path=data + +# Use direct uploads +# WARNING: disables resumable uploads, files are uploaded in a single request +# This may help prevent IOException: insufficient data written, +# see https://github.com/pinterest/secor/issues/177 +# https://cloud.google.com/storage/docs/json_api/v1/how-tos/upload +secor.gs.upload.direct=false + +# Application credentials configuration file +# https://developers.google.com/identity/protocols/application-default-credentials +# It can be empty when secor running in Google Cloud VMs with proper scopes +secor.gs.credentials.path= + # Zookeeper config. zookeeper.session.timeout.ms=3000 zookeeper.sync.time.ms=200 +# Zookeeper path (chroot) under which secor data will be placed. +secor.zookeeper.path=/ + # Impacts how frequently the upload logic is triggered if no messages are delivered. kafka.consumer.timeout.ms=10000 +# Where consumer should read from if no committed offset in zookeeper. +# "smallest" -> read from earliest offset +# "largest" -> read from latest offset +# Always use "smallest" unless you know what you're doing and are willing to risk +# data loss for new topics or topics whose number of partitions has changed. +# See the kafka docs for "auto.offset.reset". +kafka.consumer.auto.offset.reset=smallest + +# Choose between range and roundrobin partition assignment strategy for kafka +# high level consumers. Check PartitionAssignor.scala in kafa 821 module for +# the differences between the two. +# In kafka 811, only range strategy is supported. +kafka.partition.assignment.strategy=range + # Max number of retries during rebalance. kafka.rebalance.max.retries= +# Rebalance backoff. +kafka.rebalance.backoff.ms= + # Kafka consumer receive buffer size (socket.receive.buffer.bytes) kafka.socket.receive.buffer.bytes= # Kafka fetch max size (fetch.message.max.bytes) kafka.fetch.message.max.bytes= +# Kafka fetch min bytes (fetch.fetch.min.bytes) +kafka.fetch.min.bytes= + +# Kafka fetch max wait ms (fetch.max.wait.ms) +kafka.fetch.wait.max.ms= + # Port of the broker serving topic partition metadata. kafka.seed.broker.port=9092 @@ -51,7 +173,28 @@ kafka.seed.broker.port=9092 # to as the chroot. kafka.zookeeper.path=/ -# Secor generation is a version that should be incremented during non-backwards-compabile +# Store offset in zookeeper and kafka consumer topic. +# Only used if kafka.offsets.storage is set to "kafka" +# http://kafka.apache.org/documentation.html#oldconsumerconfigs +# Possible values: true or false +kafka.dual.commit.enabled=true + +# Storage offset. +# Possible values: "zookeeper" to read offset from zookeeper or "kafka" to read offset from kafka consumer topic +kafka.offsets.storage=zookeeper + +# Parameter which tells whether to extract Kafka message timestamp. This value is to be chose in case of 0.10.x kafka brokers. +# Default value is false. Also specify `kafka.message.timestamp.className` as `com.pinterest.secor.timestamp.Kafka10MessageTimestamp`, +# in case you are enabling this parameter as `true`. +kafka.useTimestamp=false + +# Classname for the timestamp field you want to use. Default is `com.pinterest.secor.timestamp.Kafka10MessageTimestamp` +# for 0.10 build profile. Basically, it will be `Kafka8MessageTimestamp` for 0.8 kafka and `Kafka10MessageTimestamp` +# for 0.10 kafka. Fully classified names are `com.pinterest.secor.timestamp.Kafka8MessageTimestamp` and +# `com.pinterest.secor.timestamp.Kafka10MessageTimestamp`. +kafka.message.timestamp.className=com.pinterest.secor.timestamp.Kafka10MessageTimestamp + +# Secor generation is a version that should be incremented during non-backwards-compatible # 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 @@ -63,18 +206,46 @@ secor.consumer.threads=7 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. +# Number of continuous 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 - +secor.offsets.prefix=offset= # How long does it take for secor to forget a topic partition. Applies to stats generation only. secor.topic_partition.forget.seconds=600 -# If greater than 0, upon starup Secor will clean up directories and files under secor.local.path +# Setting the partitioner to use hourly partition +# By default, the partitioner will do daily partition, so the data will be +# written into +# s3n://.../topic/dt=2015-07-07/ +# If this parameter is set to true, the data will be written into +# s3n://.../topic/dt=2015-07-07/hr=02 +# The hour folder ranges from 00 to 23 +partitioner.granularity.hour=false +partitioner.granularity.minute=false + +partitioner.granularity.date.prefix=dt= +partitioner.granularity.hour.prefix=hr= +partitioner.granularity.minute.prefix=min= + +partitioner.granularity.date.format=yyyy-MM-dd +partitioner.granularity.hour.format=HH +partitioner.granularity.minute.format=mm + +# how many seconds should the finalizer wait to finalize a partition +partitioner.finalizer.delay.seconds=3600 + +# During partition finalization, the finalizer will start from the last +# time partition (e.g. dt=2015-07-17) and traverse backwards for n +# partition periods (e.g. dt=2015-07-16, dt=2015-07-15 ...) +# This parameter controls how many partition periods to traverse back +# The default is 10 +# secor.finalizer.lookback.periods=10 + +# If greater than 0, upon startup Secor will clean up directories and files under secor.local.path # that are older than this value. secor.local.log.delete.age.hours=-1 @@ -83,19 +254,167 @@ secor.local.log.delete.age.hours=-1 # It is available at https://api.qubole.com/users/edit qubole.api.token= +# hive tables are generally named after the topics. For instance if the topic +# is request_log the hive table is also called request_log. If you want this +# to be pinlog_request_log you can set this config to "pinlog_". This affects +# all topics. +hive.table.prefix= + +# You can also name your hive table directly if your hive table doesn't +# follow the pattern of +# E.g. hive.table.name.topic1=table1 to indicate that hive table for +# kafka topic will be named + # Secor can export stats such as consumption lag (in seconds and offsets) per topic partition. # Leave empty to disable this functionality. tsdb.hostport= # Regex of topics that are not exported to TSDB. -tsdb.blacklist.topics= +monitoring.blacklist.topics= + +# Prefix of exported stats. +monitoring.prefix=secor + +# Monitoring interval. +# Set to 0 to disable - the progress monitor will run once and exit. +monitoring.interval.seconds=0 -# Name of field that contains timestamp for JSON or Thrift message parser. (1405970352123) +# Secor can export stats to statsd such as consumption lag (in seconds and offsets) per topic partition. +# Leave empty to disable this functionality. +statsd.hostport= + +# Thrift protocol class. It applies to timestamp extractor below and parquet output for thrift messages. +# TBinaryProtocol by default +secor.thrift.protocol.class= + +# Thrift message class. It applies to parquet output. +# If all Kafka topics transfer the same thrift message type, set secor.thrift.message.class.*= +secor.thrift.message.class.*= + +# If true, the consumer group will be the initial prefix of all +# exported metrics, before `monitoring.prefix` (if set). +# +# Setting to false and use monitoring.prefix can lead to nice paths. +# For example, +# secor.kafka.group = secor_hr_partition +# monitoring.prefix = secor.hr +# statsd.prefixWithConsumerGroup = false +# => secor.hr.lag.offsets.. +# +# secor.kafka.group = secor_hr_partition +# monitoring.prefix = secor +# statsd.prefixWithConsumerGroup = true +# => secor_hr_partition.secor.lag.offsets.. +statsd.prefixWithConsumerGroup=true + +# Name of field that contains timestamp for JSON, MessagePack, or Thrift message parser. (1405970352123) message.timestamp.name=timestamp +# Separator for defining message.timestamp.name in a nested structure. E.g. +# {"meta_data": {"created": "1405911096123", "last_modified": "1405912096123"}, "data": "test"} +# message.timestamp.name=meta_data.created +# message.timestamp.name.separator=. +message.timestamp.name.separator= + +# Field ID of the field that contains timestamp for Thrift message parser. +# N.B. setting this past 1 will come with a performance penalty +message.timestamp.id=1 + +# Data type of the timestamp field for thrift message parser. +# Supports i64 and i32. +message.timestamp.type=i64 + # Name of field that contains a timestamp, as a date Format, for JSON. (2014-08-07, Jul 23 02:16:57 2005, etc...) # Should be used when there is no timestamp in a Long format. Also ignore time zones. message.timestamp.input.pattern= -# To enable compression, set this to a valid compression codec, such as 'org.apache.hadoop.io.compress.GzipCodec'. +# whether timestamp field is required, it should always be required. But +# for historical reason, we didn't enforce this check, there might exist some +# installations with messages missing timestamp field +message.timestamp.required=true + +# To enable compression, set this to a valid compression codec implementing +# org.apache.hadoop.io.compress.CompressionCodec interface, such as +# 'org.apache.hadoop.io.compress.GzipCodec'. secor.compression.codec= + +# To set a custom file extension set this to a valid file suffix, such as +# '.gz', '.part', etc. +secor.file.extension= + +# The secor file reader/writer used to read/write the data, by default we write sequence files +secor.file.reader.writer.factory=com.pinterest.secor.io.impl.SequenceFileReaderWriterFactory +#if left blank defaults to \n +secor.file.reader.Delimiter=\n +#if left blank no Delimiter is added. do not use \ as that needs to be escaped and is an escape +#character and not a delimtier. +secor.file.writer.Delimiter=\n + +# Max message size in bytes to retrieve via KafkaClient. This is used by ProgressMonitor and PartitionFinalizer. +# This should be set large enough to accept the max message size configured in your kafka broker +# Default is 0.1 MB +secor.max.message.size.bytes=100000 + +# Class that will manage uploads. Default is to use the hadoop +# interface to S3. +secor.upload.manager.class=com.pinterest.secor.uploader.HadoopS3UploadManager + +#Set below property to your timezone, and partitions in s3 will be created as per timezone provided +secor.parser.timezone=UTC + +# Transformer class that transform message accordingly. +secor.message.transformer.class=com.pinterest.secor.transformer.IdentityMessageTransformer + +# Set below property to true if you want to have the md5hash appended to your s3 path. +# This helps for better partitioning of the data on s3. Which gives better performance while reading and writing on s3 +secor.s3.prefix.md5hash=false + +# After the given date, secor will upload files to the supplied s3 alternative path +secor.s3.alter.path.date= + +# An alternative S3 path for secor to upload files to +secor.s3.alternative.path= + +# If enabled, add calls will be made to qubole, otherwise, skip qubole call for finalization +secor.enable.qubole=true + +# Timeout value for qubole calls +secor.qubole.timeout.ms=300000 + +# Topics to upload at a fixed minute mark +secor.kafka.upload_at_minute_mark.topic_filter= + +# What the minute mark is. This isn't triggered unless the topic name matches +secor.upload.minute_mark=0 + +# File age per topic and per partition is checked against secor.max.file.age.seconds by looking at +# the youngest file when true or at the oldest file when false. +secor.file.age.youngest=true + +# Class that manages metric collection. +# Sending metrics to Ostrich is the default implementation. +secor.monitoring.metrics.collector.class=com.pinterest.secor.monitoring.OstrichMetricCollector + +# Row group size in bytes for Parquet writers. Specifies how much data will be buffered in memory before flushing a +# block to disk. Larger values allow for larger column chinks which makes it possible to do larger sequential IO. +# Should be aligned with HDFS blocks. Defaults to 128MB in Parquet 1.9. +parquet.block.size=134217728 + +# Page group size in bytes for Parquet writers. Indivisible unit for columnar data. Smaller data pages allow for more +# fine grained reading but have higher space overhead. Defaults to 1MB in Parquet 1.9. +parquet.page.size=1048576 + +# Enable or disable dictionary encoding for Parquet writers. The dictionary encoding builds a dictionary of values +# encountered in a given column. Defaults to true in Parquet 1.9. +parquet.enable.dictionary=true + +# Enable or disable validation for Parquet writers. Validates records written against the schema. Defaults to false in +# Parquet 1.9. +parquet.validation=false + +# User can configure ORC schema for each Kafka topic. Common schema is also possible. This property is mandatory +# if DefaultORCSchemaProvider is used. ORC schema for all the topics should be defined like this: +secor.orc.message.schema.*=struct\,f:array\,g:int> +# Below config used for defining ORC schema provider class name. User can use the custom implementation for orc schema provider +secor.orc.schema.provider=com.pinterest.secor.util.orc.schema.DefaultORCSchemaProvider + diff --git a/src/main/config/secor.dev.azure.properties b/src/main/config/secor.dev.azure.properties new file mode 100644 index 000000000..c9de7bc4e --- /dev/null +++ b/src/main/config/secor.dev.azure.properties @@ -0,0 +1,28 @@ +include=secor.dev.properties + +# Configure upload manager class to use Azure blob storage upload manager +secor.upload.manager.class=com.pinterest.secor.uploader.AzureUploadManager + +############ +# MUST SET # +############ + +# Microsoft Azure blob storage default endpoint protocol +secor.azure.endpoints.protocol=https + +# Microsoft Azure authentication credentials. +# https://azure.microsoft.com/en-us/documentation/articles/storage-create-storage-account +secor.azure.account.name= +secor.azure.account.key= + +# Microsoft Azure blob storage container name. Container is a grouping of a set +# of blobs. https://msdn.microsoft.com/en-us/library/dd135715.aspx +secor.azure.container.name= + +# Microsoft Azure blob storage path where files are stored within the container. +secor.azure.path=data + +################ +# END MUST SET # +################ + diff --git a/src/main/config/secor.dev.backup.properties b/src/main/config/secor.dev.backup.properties index fac8845d5..858b76075 100644 --- a/src/main/config/secor.dev.backup.properties +++ b/src/main/config/secor.dev.backup.properties @@ -24,6 +24,9 @@ secor.message.parser.class=com.pinterest.secor.parser.OffsetMessageParser # S3 path where sequence files are stored. secor.s3.path=secor_dev/backup +# Swift path where sequence files are stored. +secor.swift.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 diff --git a/src/main/config/secor.dev.gs.properties b/src/main/config/secor.dev.gs.properties new file mode 100644 index 000000000..5eb765bb1 --- /dev/null +++ b/src/main/config/secor.dev.gs.properties @@ -0,0 +1,29 @@ +include=secor.dev.properties + +# Configure upload manager class to use Google Storage Upload Manager +secor.upload.manager.class=com.pinterest.secor.uploader.GsUploadManager + +############ +# MUST SET # +############ + +# Name of the Google cloud storage bucket where log files are stored. +secor.gs.bucket=secor_gs + +# Google cloud storage path where files are stored within the bucket. +secor.gs.path=data + +################ +# END MUST SET # +################ + +# Application credentials configuration file +# https://developers.google.com/identity/protocols/application-default-credentials +secor.gs.credentials.path=google_app_credentials.json + +# Use direct uploads +# WARNING: disables resumable uploads, files are uploaded in a single request +# This may help prevent IOException: insufficient data written, +# see https://github.com/pinterest/secor/issues/177 +# https://cloud.google.com/storage/docs/json_api/v1/how-tos/upload +secor.gs.upload.direct=false diff --git a/src/main/config/secor.dev.hr.partition.properties b/src/main/config/secor.dev.hr.partition.properties new file mode 100644 index 000000000..95c7017c2 --- /dev/null +++ b/src/main/config/secor.dev.hr.partition.properties @@ -0,0 +1,11 @@ +include=secor.dev.properties + +secor.kafka.group=secor_hr_partition +secor.message.parser.class=com.pinterest.secor.parser.ThriftMessageParser + +secor.s3.path=secor_dev/hr_partition +secor.local.path=/tmp/secor_dev/message_logs/hr_partition + +partitioner.granularity.hour=true + +ostrich.port=9998 diff --git a/src/main/config/secor.dev.partition.properties b/src/main/config/secor.dev.partition.properties index d029c86d0..d91ff69b1 100644 --- a/src/main/config/secor.dev.partition.properties +++ b/src/main/config/secor.dev.partition.properties @@ -24,6 +24,9 @@ secor.message.parser.class=com.pinterest.secor.parser.ThriftMessageParser # S3 path where sequence files are stored. secor.s3.path=secor_dev/partition +# Swift path where sequence files are stored. +secor.swift.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 diff --git a/src/main/config/secor.dev.properties b/src/main/config/secor.dev.properties index b1ad32099..636ea44be 100644 --- a/src/main/config/secor.dev.properties +++ b/src/main/config/secor.dev.properties @@ -4,9 +4,26 @@ include=secor.common.properties # MUST SET # ############ +# Fill the section which fits your needs +############### +# Using S3 # +############### + # Name of the s3 bucket where log files are stored. secor.s3.bucket= +############### +# Using Swift # +############### + +# Boolean variable which determines if each topic will be uploaded to different container +# The Containers for the topics will be Created automatically +# If true, then the setting "secor.swift.container" will be ignored +secor.swift.containers.for.each.topic=false + +# Name of swift container where log files are stored. +secor.swift.container=logsContainer + ################ # END MUST SET # ################ @@ -19,6 +36,6 @@ zookeeper.quorum=localhost:2181 # Upload policies. # 10K secor.max.file.size.bytes=10000 -# 1 minute -secor.max.file.age.seconds=60 +# 10 seconds +secor.max.file.age.seconds=10 diff --git a/src/main/config/secor.prod.backup.properties b/src/main/config/secor.prod.backup.properties index 35f57f0e8..606cf5626 100644 --- a/src/main/config/secor.prod.backup.properties +++ b/src/main/config/secor.prod.backup.properties @@ -18,12 +18,15 @@ include=secor.prod.properties # Name of the Kafka consumer group. secor.kafka.group=secor_backup -# Parser class that extracts s3 partitions from consumed messages. +# Parser class that extracts 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 +# Swift path where sequence files are stored. +secor.swift.path= + # Local path where sequence files are stored before they are uploaded to s3. secor.local.path=/mnt/secor_data/message_logs/backup diff --git a/src/main/config/secor.prod.partition.properties b/src/main/config/secor.prod.partition.properties index 14e36e667..baf66d4ee 100644 --- a/src/main/config/secor.prod.partition.properties +++ b/src/main/config/secor.prod.partition.properties @@ -24,6 +24,9 @@ secor.message.parser.class=com.pinterest.secor.parser.ThriftMessageParser # S3 path where sequence files are stored. secor.s3.path=raw_logs +# Swift path where sequence files are stored. +secor.swift.path= + # Local path where sequence files are stored before they are uploaded to s3. secor.local.path=/mnt/secor_data/message_logs/partition diff --git a/src/main/config/secor.prod.properties b/src/main/config/secor.prod.properties index 0801a5c99..8ce4718f8 100644 --- a/src/main/config/secor.prod.properties +++ b/src/main/config/secor.prod.properties @@ -19,16 +19,32 @@ 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. +# List of Kafka Zookeeper servers with ports :. zookeeper.quorum= +# Fill the section which fits your needs +############### +# Using S3 # +############### + +# Name of the s3 bucket where log files are stored. +secor.s3.bucket= + +############### +# Using Swift # +############### + +# Boolean variable which determines if each topic will be uploaded to different container +# (Created automatic) - if true the next setting will be ignored +secor.swift.containers.for.each.topic=false + +# Name of swift container where log files are stored. +secor.swift.container=logsContainer + ################ # END MUST SET # ################ @@ -37,5 +53,6 @@ zookeeper.quorum= # 200MB secor.max.file.size.bytes=200000000 # 1 hour +# for hourly ingestion/finalization, set this property to smaller value, e.g. 1800 secor.max.file.age.seconds=3600 diff --git a/src/main/config/secor.test.backup.properties b/src/main/config/secor.test.backup.properties new file mode 100644 index 000000000..4d9f0a531 --- /dev/null +++ b/src/main/config/secor.test.backup.properties @@ -0,0 +1,3 @@ +include=secor.test.properties +include=secor.dev.backup.properties + diff --git a/src/main/config/secor.test.partition.properties b/src/main/config/secor.test.partition.properties new file mode 100644 index 000000000..c902e15fc --- /dev/null +++ b/src/main/config/secor.test.partition.properties @@ -0,0 +1,3 @@ +include=secor.test.properties +include=secor.dev.partition.properties + diff --git a/src/main/config/secor.test.perf.backup.properties b/src/main/config/secor.test.perf.backup.properties new file mode 100644 index 000000000..3fd77f9b2 --- /dev/null +++ b/src/main/config/secor.test.perf.backup.properties @@ -0,0 +1,19 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +include=secor.dev.backup.properties + +#The prefix used by the topics created for the perf test +secor.kafka.perf_topic_prefix= \ No newline at end of file diff --git a/src/main/config/secor.test.properties b/src/main/config/secor.test.properties new file mode 100644 index 000000000..1149b8120 --- /dev/null +++ b/src/main/config/secor.test.properties @@ -0,0 +1,4 @@ +secor.s3.bucket=test-bucket +aws.access.key=TESTKEY +aws.secret.key=TESTKEY + diff --git a/src/main/java/com/pinterest/secor/common/FileRegistry.java b/src/main/java/com/pinterest/secor/common/FileRegistry.java index f405cc28d..eb67e0213 100644 --- a/src/main/java/com/pinterest/secor/common/FileRegistry.java +++ b/src/main/java/com/pinterest/secor/common/FileRegistry.java @@ -16,14 +16,11 @@ */ package com.pinterest.secor.common; +import com.pinterest.secor.io.FileWriter; import com.pinterest.secor.util.FileUtil; +import com.pinterest.secor.util.ReflectionUtil; 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.apache.hadoop.io.compress.CompressionCodec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,13 +37,15 @@ public class FileRegistry { private static final Logger LOG = LoggerFactory.getLogger(FileRegistry.class); - private HashMap> mFiles; - private HashMap mWriters; + private final SecorConfig mConfig; + private HashMap> mFiles; + private HashMap mWriters; private HashMap mCreationTimes; - public FileRegistry() { - mFiles = new HashMap>(); - mWriters = new HashMap(); + public FileRegistry(SecorConfig mConfig) { + this.mConfig = mConfig; + mFiles = new HashMap>(); + mWriters = new HashMap(); mCreationTimes = new HashMap(); } @@ -55,12 +54,23 @@ public FileRegistry() { * @return Collection of all registered topic partitions. */ public Collection getTopicPartitions() { - Set topicPartitions = mFiles.keySet(); - if (topicPartitions == null) { - return new HashSet(); + Collection topicPartitions = getTopicPartitionGroups(); + Set tps = new HashSet(); + if (topicPartitions != null) { + for (TopicPartitionGroup g : topicPartitions) { + tps.addAll(g.getTopicPartitions()); + } + } + return tps; + } + + public Collection getTopicPartitionGroups() { + Set topicPartitions = mFiles.keySet(); + Set tps = new HashSet(); + if (topicPartitions != null) { + tps.addAll(topicPartitions); } - // Return a copy of the collection to prevent the caller from modifying internals. - return new HashSet(topicPartitions); + return tps; } /** @@ -69,28 +79,48 @@ public Collection getTopicPartitions() { * @return Collection of file paths in the given topic partition. */ public Collection getPaths(TopicPartition topicPartition) { - HashSet logFilePaths = mFiles.get(topicPartition); + return getPaths(new TopicPartitionGroup(topicPartition)); + } + + /** + * Get paths in a given topic partition. + * @param topicPartitionGroup The topic partition to retrieve paths for. + * @return Collection of file paths in the given topic partition. + */ + public Collection getPaths(TopicPartitionGroup topicPartitionGroup) { + HashSet logFilePaths = mFiles.get(topicPartitionGroup); if (logFilePaths == null) { return new HashSet(); } return new HashSet(logFilePaths); } + /** + * Retrieve an existing writer for a given path. + * @param path The path to retrieve writer for. + * @return Writer for a given path or null if no writer has been created yet. + */ + public FileWriter getWriter(LogFilePath path) + throws Exception { + return mWriters.get(path); + } + /** * 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. * @param codec Optional compression codec. * @return Writer for a given path. - * @throws IOException + * @throws Exception */ - public SequenceFile.Writer getOrCreateWriter(LogFilePath path, CompressionCodec codec) throws IOException { - SequenceFile.Writer writer = mWriters.get(path); + public FileWriter getOrCreateWriter(LogFilePath path, CompressionCodec codec) + throws Exception { + FileWriter 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()); + TopicPartitionGroup topicPartition = new TopicPartitionGroup(path.getTopic(), + path.getKafkaPartitions()); HashSet files = mFiles.get(topicPartition); if (files == null) { files = new HashSet(); @@ -99,21 +129,15 @@ public SequenceFile.Writer getOrCreateWriter(LogFilePath path, CompressionCodec if (!files.contains(path)) { files.add(path); } - Configuration config = new Configuration(); - FileSystem fs = FileSystem.get(config); - if (codec != null) { - Path fsPath = new Path(path.getLogFilePath()); - writer = SequenceFile.createWriter(fs, config, fsPath, LongWritable.class, - BytesWritable.class, - SequenceFile.CompressionType.BLOCK, codec); - } else { - Path fsPath = new Path(path.getLogFilePath()); - writer = SequenceFile.createWriter(fs, config, fsPath, LongWritable.class, - BytesWritable.class); - } + writer = ReflectionUtil.createFileWriter(mConfig.getFileReaderWriterFactory(), path, codec, mConfig); mWriters.put(path, writer); mCreationTimes.put(path, System.currentTimeMillis() / 1000L); - LOG.debug("created writer for path " + path.getLogFilePath()); + LOG.debug("created writer for path {}", path.getLogFilePath()); + LOG.debug("Register deleteOnExit for path {}", path.getLogFilePath()); + FileUtil.deleteOnExit(path.getLogFileParentDir()); + FileUtil.deleteOnExit(path.getLogFileDir()); + FileUtil.deleteOnExit(path.getLogFilePath()); + FileUtil.deleteOnExit(path.getLogFileCrcPath()); } return writer; } @@ -124,16 +148,16 @@ public SequenceFile.Writer getOrCreateWriter(LogFilePath path, CompressionCodec * @throws IOException */ public void deletePath(LogFilePath path) throws IOException { - TopicPartition topicPartition = new TopicPartition(path.getTopic(), - path.getKafkaPartition()); + TopicPartitionGroup topicPartition = new TopicPartitionGroup(path.getTopic(), + path.getKafkaPartitions()); HashSet paths = mFiles.get(topicPartition); paths.remove(path); if (paths.isEmpty()) { mFiles.remove(topicPartition); StatsUtil.clearLabel("secor.size." + topicPartition.getTopic() + "." + - topicPartition.getPartition()); + topicPartition.getPartitions()[0]); StatsUtil.clearLabel("secor.modification_age_sec." + topicPartition.getTopic() + "." + - topicPartition.getPartition()); + topicPartition.getPartitions()[0]); } deleteWriter(path); FileUtil.delete(path.getLogFilePath()); @@ -146,7 +170,11 @@ public void deletePath(LogFilePath path) throws IOException { * @throws IOException */ public void deleteTopicPartition(TopicPartition topicPartition) throws IOException { - HashSet paths = mFiles.get(topicPartition); + deleteTopicPartitionGroup((new TopicPartitionGroup(topicPartition))); + } + + public void deleteTopicPartitionGroup(TopicPartitionGroup topicPartitioGroup) throws IOException { + HashSet paths = mFiles.get(topicPartitioGroup); if (paths == null) { return; } @@ -161,11 +189,11 @@ public void deleteTopicPartition(TopicPartition topicPartition) throws IOExcepti * @param path The path to remove the writer for. */ public void deleteWriter(LogFilePath path) throws IOException { - SequenceFile.Writer writer = mWriters.get(path); + FileWriter writer = mWriters.get(path); if (writer == null) { - LOG.warn("No writer found for path " + path.getLogFilePath()); + LOG.warn("No writer found for path {}", path.getLogFilePath()); } else { - LOG.info("Deleting writer for path " + path.getLogFilePath()); + LOG.info("Deleting writer for path {}", path.getLogFilePath()); writer.close(); mWriters.remove(path); mCreationTimes.remove(path); @@ -177,10 +205,14 @@ public void deleteWriter(LogFilePath path) throws IOException { * @param topicPartition The topic partition to remove the writers for. */ public void deleteWriters(TopicPartition topicPartition) throws IOException { - HashSet paths = mFiles.get(topicPartition); + deleteWriters(new TopicPartitionGroup(topicPartition)); + } + + public void deleteWriters(TopicPartitionGroup topicPartitionGroup) throws IOException { + HashSet paths = mFiles.get(topicPartitionGroup); if (paths == null) { - LOG.warn("No paths found for topic " + topicPartition.getTopic() + " partition " + - topicPartition.getPartition()); + LOG.warn("No paths found for topic {} partition {}", topicPartitionGroup.getTopic(), + Arrays.toString(topicPartitionGroup.getPartitions())); } else { for (LogFilePath path : paths) { deleteWriter(path); @@ -196,16 +228,20 @@ public void deleteWriters(TopicPartition topicPartition) throws IOException { * @throws IOException */ public long getSize(TopicPartition topicPartition) throws IOException { - Collection paths = getPaths(topicPartition); + return getSize(new TopicPartitionGroup(topicPartition)); + } + + public long getSize(TopicPartitionGroup topicPartitionGroup) throws IOException { + Collection paths = getPaths(topicPartitionGroup); long result = 0; for (LogFilePath path : paths) { - SequenceFile.Writer writer = mWriters.get(path); + FileWriter writer = mWriters.get(path); if (writer != null) { result += writer.getLength(); } } - StatsUtil.setLabel("secor.size." + topicPartition.getTopic() + "." + - topicPartition.getPartition(), Long.toString(result)); + StatsUtil.setLabel("secor.size." + topicPartitionGroup.getTopic() + "." + + Arrays.toString(topicPartitionGroup.getPartitions()), Long.toString(result)); return result; } @@ -217,25 +253,40 @@ public long getSize(TopicPartition topicPartition) throws IOException { * @throws IOException */ public long getModificationAgeSec(TopicPartition topicPartition) throws IOException { + return getModificationAgeSec(new TopicPartitionGroup(topicPartition)); + } + + public long getModificationAgeSec(TopicPartitionGroup topicPartitionGroup) throws IOException { long now = System.currentTimeMillis() / 1000L; - long result = Long.MAX_VALUE; - Collection paths = getPaths(topicPartition); + long result; + if (mConfig.getFileAgeYoungest()) { + result = Long.MAX_VALUE; + } else { + result = -1; + } + Collection paths = getPaths(topicPartitionGroup); for (LogFilePath path : paths) { Long creationTime = mCreationTimes.get(path); if (creationTime == null) { - LOG.warn("no creation time found for path " + path); + LOG.warn("no creation time found for path {}", path); creationTime = now; } long age = now - creationTime; - if (age < result) { - result = age; + if (mConfig.getFileAgeYoungest()) { + if (age < result) { + result = age; + } + } else { + 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)); + StatsUtil.setLabel("secor.modification_age_sec." + topicPartitionGroup.getTopic() + "." + + Arrays.toString(topicPartitionGroup.getPartitions()), 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 index 3fe96a268..1e1a20ae9 100644 --- a/src/main/java/com/pinterest/secor/common/KafkaClient.java +++ b/src/main/java/com/pinterest/secor/common/KafkaClient.java @@ -18,7 +18,9 @@ import com.google.common.net.HostAndPort; import com.pinterest.secor.message.Message; -import kafka.api.*; +import com.pinterest.secor.timestamp.KafkaMessageTimestampFactory; +import kafka.api.FetchRequestBuilder; +import kafka.api.PartitionOffsetRequestInfo; import kafka.common.TopicAndPartition; import kafka.javaapi.FetchResponse; import kafka.javaapi.OffsetRequest; @@ -29,6 +31,7 @@ import kafka.javaapi.TopicMetadataResponse; import kafka.javaapi.consumer.SimpleConsumer; import kafka.message.MessageAndOffset; +import org.apache.kafka.common.protocol.Errors; import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,20 +52,24 @@ public class KafkaClient { private SecorConfig mConfig; private ZookeeperConnector mZookeeperConnector; + private KafkaMessageTimestampFactory mKafkaMessageTimestampFactory; public KafkaClient(SecorConfig config) { mConfig = config; mZookeeperConnector = new ZookeeperConnector(mConfig); + mKafkaMessageTimestampFactory = new KafkaMessageTimestampFactory(mConfig.getKafkaMessageTimestampClass()); } + public class MessageDoesNotExistException extends RuntimeException {} + 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"); + LOG.debug("looking up leader for topic {} partition {}", topicPartition.getTopic(), topicPartition.getPartition()); + consumer = createConsumer( + mConfig.getKafkaSeedBrokerHost(), + mConfig.getKafkaSeedBrokerPort(), + "leaderLookup"); List topics = new ArrayList(); topics.add(topicPartition.getTopic()); TopicMetadataRequest request = new TopicMetadataRequest(topics); @@ -112,9 +119,9 @@ private long findLastOffset(TopicPartition topicPartition, SimpleConsumer consum 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; + LOG.debug("fetching message topic {} partition {} offset {}", + topicPartition.getTopic(), topicPartition.getPartition(), offset); + final int MAX_MESSAGE_SIZE_BYTES = mConfig.getMaxMessageSizeBytes(); final String clientName = getClientName(topicPartition); kafka.api.FetchRequest request = new FetchRequestBuilder().clientId(clientName) .addFetch(topicPartition.getTopic(), topicPartition.getPartition(), offset, @@ -123,33 +130,54 @@ private Message getMessage(TopicPartition topicPartition, long offset, FetchResponse response = consumer.fetch(request); if (response.hasError()) { consumer.close(); - throw new RuntimeException("Error fetching offset data. Reason: " + - response.errorCode(topicPartition.getTopic(), topicPartition.getPartition())); + int errorCode = response.errorCode(topicPartition.getTopic(), topicPartition.getPartition()); + + if (errorCode == Errors.OFFSET_OUT_OF_RANGE.code()) { + throw new MessageDoesNotExistException(); + } else { + throw new RuntimeException("Error fetching offset data. Reason: " + errorCode); + } } MessageAndOffset messageAndOffset = response.messageSet( topicPartition.getTopic(), topicPartition.getPartition()).iterator().next(); - ByteBuffer payload = messageAndOffset.message().payload(); - byte[] payloadBytes = new byte[payload.limit()]; - payload.get(payloadBytes); + byte[] keyBytes = null; + if (messageAndOffset.message().hasKey()) { + ByteBuffer key = messageAndOffset.message().key(); + keyBytes = new byte[key.limit()]; + key.get(keyBytes); + } + byte[] payloadBytes = null; + if (!messageAndOffset.message().isNull()) { + ByteBuffer payload = messageAndOffset.message().payload(); + payloadBytes = new byte[payload.limit()]; + payload.get(payloadBytes); + } + long timestamp = (mConfig.useKafkaTimestamp()) + ? mKafkaMessageTimestampFactory.getKafkaMessageTimestamp().getTimestamp(messageAndOffset) + : 0l; + return new Message(topicPartition.getTopic(), topicPartition.getPartition(), - messageAndOffset.offset(), payloadBytes); + messageAndOffset.offset(), keyBytes, payloadBytes, timestamp); + } + + private SimpleConsumer createConsumer(String host, int port, String clientName) { + return new SimpleConsumer(host, port, 100000, 64 * 1024, clientName); } - private SimpleConsumer createConsumer(TopicPartition topicPartition) { + public SimpleConsumer createConsumer(TopicPartition topicPartition) { HostAndPort leader = findLeader(topicPartition); - LOG.info("leader for topic " + topicPartition.getTopic() + " partition " + - topicPartition.getPartition() + " is " + leader.toString()); + LOG.debug("leader for topic {} partition {} is {}", topicPartition.getTopic(), topicPartition.getPartition(), leader.toString()); final String clientName = getClientName(topicPartition); - return new SimpleConsumer(leader.getHostText(), leader.getPort(), 100000, 64 * 1024, - clientName); + return createConsumer(leader.getHostText(), leader.getPort(), clientName); } public int getNumPartitions(String topic) { SimpleConsumer consumer = null; try { - consumer = new SimpleConsumer(mConfig.getKafkaSeedBrokerHost(), - mConfig.getKafkaSeedBrokerPort(), - 100000, 64 * 1024, "partitionLookup"); + consumer = createConsumer( + mConfig.getKafkaSeedBrokerHost(), + mConfig.getKafkaSeedBrokerPort(), + "partitionLookup"); List topics = new ArrayList(); topics.add(topic); TopicMetadataRequest request = new TopicMetadataRequest(topics); @@ -168,20 +196,45 @@ public int getNumPartitions(String topic) { } public Message getLastMessage(TopicPartition topicPartition) throws TException { - SimpleConsumer consumer = createConsumer(topicPartition); - long lastOffset = findLastOffset(topicPartition, consumer); - if (lastOffset < 1) { - return null; + SimpleConsumer consumer = null; + try { + consumer = createConsumer(topicPartition); + long lastOffset = findLastOffset(topicPartition, consumer); + if (lastOffset < 1) { + return null; + } + return getMessage(topicPartition, lastOffset, consumer); + } finally { + if (consumer != null) { + consumer.close(); + } } - 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 = null; + try { + long committedOffset = mZookeeperConnector.getCommittedOffsetCount(topicPartition) - 1; + if (committedOffset < 0) { + return null; + } + consumer = createConsumer(topicPartition); + return getMessage(topicPartition, committedOffset, consumer); + } catch (MessageDoesNotExistException e) { + // If a RuntimeEMessageDoesNotExistException exception is raised, + // the message at the last comitted offset does not exist in Kafka. + // This is usually due to the message being compacted away by the + // Kafka log compaction process. + // + // That is no an exceptional situation - in fact it can be normal if + // the topic being consumed by Secor has a low volume. So in that + // case, simply return null + LOG.warn("no committed message for topic {} partition {}", topicPartition.getTopic(), topicPartition.getPartition()); + return null; + } finally { + if (consumer != null) { + consumer.close(); + } } - 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 index 367dc03f4..e071b3d91 100644 --- a/src/main/java/com/pinterest/secor/common/LogFilePath.java +++ b/src/main/java/com/pinterest/secor/common/LogFilePath.java @@ -17,17 +17,22 @@ package com.pinterest.secor.common; import com.pinterest.secor.message.ParsedMessage; +import org.apache.commons.codec.binary.Base64; import org.apache.commons.lang.StringUtils; +import java.io.UnsupportedEncodingException; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Map; /** * 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 + * prefix/topic/partition1/.../partitionN/generation_kafkaPartition_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, @@ -42,42 +47,56 @@ * @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; - private String mExtension; + private final String mPrefix; + private final String mTopic; + private final String[] mPartitions; + private final int mGeneration; + private final int[] mKafkaPartitions; + private final long[] mOffsets; + private final String mExtension; + private Map customTopicsNames; + private MessageDigest messageDigest; - public LogFilePath(String prefix, int generation, long lastCommittedOffset, - ParsedMessage message, String extension) { - mPrefix = prefix; - mTopic = message.getTopic(); - mPartitions = message.getPartitions(); - mGeneration = generation; - mKafkaPartition = message.getKafkaPartition(); - mOffset = lastCommittedOffset; - mExtension = extension; - } public LogFilePath(String prefix, String topic, String[] partitions, int generation, - int kafkaPartition, long offset, String extension) { + int[] kafkaPartitions, long[] offsets, String extension) { + assert kafkaPartitions != null & kafkaPartitions.length >= 1 + : "Wrong kafkaParttions: " + Arrays.toString(kafkaPartitions); + assert offsets != null & offsets.length >= 1 : "Wrong offsets: " + Arrays.toString(offsets); + assert kafkaPartitions.length == offsets.length + : "Size mismatch partitions: " + Arrays.toString(kafkaPartitions) + + " offsets: " + Arrays.toString(offsets); + for (int i = 1; i < kafkaPartitions.length; i++) { + assert kafkaPartitions[i] == kafkaPartitions[i - 1] + 1 + : "Non consecutive partitions " + kafkaPartitions[i] + + " and " + kafkaPartitions[i-1]; + } mPrefix = prefix; mTopic = topic; - mPartitions = partitions; + mPartitions = Arrays.copyOf(partitions, partitions.length); mGeneration = generation; - mKafkaPartition = kafkaPartition; - mOffset = offset; + mKafkaPartitions = Arrays.copyOf(kafkaPartitions, kafkaPartitions.length); + mOffsets = Arrays.copyOf(offsets, offsets.length); mExtension = extension; - } - 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]; + try { + messageDigest = MessageDigest.getInstance("MD5"); + } catch (NoSuchAlgorithmException e) { + throw new RuntimeException("Unable to find mdt digest.", e); } - return result; + } + + public LogFilePath(String prefix, int generation, long lastCommittedOffset, + ParsedMessage message, String extension) { + this(prefix, message.getTopic(), message.getPartitions(), generation, + new int[]{message.getKafkaPartition()}, new long[]{lastCommittedOffset}, + extension); + } + + public LogFilePath(String prefix, String topic, String[] partitions, int generation, + int kafkaPartition, long offset, String extension) { + this(prefix, topic, partitions, generation, new int[]{kafkaPartition}, + new long[]{offset}, extension); } public LogFilePath(String prefix, String path) { @@ -110,17 +129,41 @@ public LogFilePath(String prefix, String path) { 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]); + mKafkaPartitions = new int[]{Integer.parseInt(basenameElements[1])}; + mOffsets = new long[]{Long.parseLong(basenameElements[2])}; + } + + 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 withPrefix(String prefix) { + return new LogFilePath(prefix, mTopic, mPartitions, mGeneration, mKafkaPartitions, mOffsets, + mExtension); } public String getLogFileParentDir() { - ArrayList elements = new ArrayList(); - elements.add(mPrefix); - elements.add(mTopic); + ArrayList elements = new ArrayList(); + if (mPrefix != null && mPrefix.length() > 0) { + elements.add(mPrefix); + } + if (mTopic != null && mTopic.length() > 0) { + if (getCustomTopicsNames() != null && + getCustomTopicsNames().isEmpty() == false && + getCustomTopicsNames().get(mTopic) != null && + getCustomTopicsNames().get(mTopic).length() > 0) { + elements.add(getCustomTopicsNames().get(mTopic)); + } else { + elements.add(mTopic); + } + } return StringUtils.join(elements, "/"); } - + public String getLogFileDir() { ArrayList elements = new ArrayList(); elements.add(getLogFileParentDir()); @@ -133,11 +176,29 @@ public String getLogFileDir() { private String getLogFileBasename() { ArrayList basenameElements = new ArrayList(); basenameElements.add(Integer.toString(mGeneration)); - basenameElements.add(Integer.toString(mKafkaPartition)); - basenameElements.add(String.format("%020d", mOffset)); + if (mKafkaPartitions.length > 1) { + String kafkaPartitions = mKafkaPartitions[0] + "-" + + mKafkaPartitions[mKafkaPartitions.length - 1]; + basenameElements.add(kafkaPartitions); + + StringBuilder sb = new StringBuilder(); + for (long offset : mOffsets) { + sb.append(offset); + } + try { + byte[] md5Bytes = messageDigest.digest(sb.toString().getBytes("UTF-8")); + byte[] encodedBytes = Base64.encodeBase64URLSafe(md5Bytes); + basenameElements.add(new String(encodedBytes)); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } else { + basenameElements.add(Integer.toString(mKafkaPartitions[0])); + basenameElements.add(String.format("%020d", mOffsets[0])); + } return StringUtils.join(basenameElements, "_"); } - + public String getLogFilePath() { String basename = getLogFileBasename(); @@ -170,17 +231,35 @@ public int getGeneration() { return mGeneration; } + @Deprecated public int getKafkaPartition() { - return mKafkaPartition; + return mKafkaPartitions[0]; } + public int[] getKafkaPartitions() { + return mKafkaPartitions; + } + + @Deprecated public long getOffset() { - return mOffset; + return mOffsets[0]; + } + + public long[] getOffsets() { + return mOffsets; } public String getExtension() { return mExtension; } + + public Map getCustomTopicsNames() { + return customTopicsNames; + } + + public void setCustomTopicsNames(Map customTopicsNames) { + this.customTopicsNames = customTopicsNames; + } @Override public boolean equals(Object o) { @@ -190,8 +269,8 @@ public boolean equals(Object o) { 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(mKafkaPartitions, that.mKafkaPartitions)) return false; + if (!Arrays.equals(mOffsets, that.mOffsets)) 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; @@ -205,8 +284,8 @@ public int hashCode() { 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)); + result = 31 * result + Arrays.hashCode(mKafkaPartitions); + result = 31 * result + Arrays.hashCode(mOffsets); return result; } diff --git a/src/main/java/com/pinterest/secor/common/OffsetTracker.java b/src/main/java/com/pinterest/secor/common/OffsetTracker.java index f67a4cc4d..4a390ca84 100644 --- a/src/main/java/com/pinterest/secor/common/OffsetTracker.java +++ b/src/main/java/com/pinterest/secor/common/OffsetTracker.java @@ -51,9 +51,13 @@ 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 (lastSeenOffset >= 0) { + LOG.warn("offset for topic {} partition {} changed from {} to {}", + topicPartition.getTopic(),topicPartition.getPartition(),lastSeenOffset, offset); + } else { + LOG.info("starting to consume topic {} partition {} from offset {}", + topicPartition.getTopic(),topicPartition.getPartition(),offset); + } } if (mFirstSeendOffset.get(topicPartition) == null) { mFirstSeendOffset.put(topicPartition, offset); diff --git a/src/main/java/com/pinterest/secor/common/SecorConfig.java b/src/main/java/com/pinterest/secor/common/SecorConfig.java index 4362d9910..2dd6d1ca0 100644 --- a/src/main/java/com/pinterest/secor/common/SecorConfig.java +++ b/src/main/java/com/pinterest/secor/common/SecorConfig.java @@ -16,12 +16,16 @@ */ package com.pinterest.secor.common; +import com.google.api.client.repackaged.com.google.common.base.Strings; import org.apache.commons.configuration.ConfigurationException; import org.apache.commons.configuration.PropertiesConfiguration; import org.apache.commons.lang.StringUtils; +import java.util.HashMap; +import java.util.Iterator; import java.util.Map; import java.util.Properties; +import java.util.TimeZone; /** * One-stop shop for Secor configuration options. @@ -31,21 +35,39 @@ public class SecorConfig { private final PropertiesConfiguration mProperties; - public static SecorConfig load() throws ConfigurationException { - // Load the default configuration file first - Properties systemProperties = System.getProperties(); - String configProperty = systemProperties.getProperty("config"); + private static final ThreadLocal mSecorConfig = new ThreadLocal() { + + @Override + protected SecorConfig initialValue() { + // Load the default configuration file first + Properties systemProperties = System.getProperties(); + String configProperty = systemProperties.getProperty("config"); + + PropertiesConfiguration properties; + try { + properties = new PropertiesConfiguration(configProperty); + } catch (ConfigurationException e) { + throw new RuntimeException("Error loading configuration from " + configProperty); + } - PropertiesConfiguration properties = new PropertiesConfiguration(configProperty); + for (final Map.Entry entry : systemProperties.entrySet()) { + properties.setProperty(entry.getKey().toString(), entry.getValue()); + } - for (final Map.Entry entry : systemProperties.entrySet()) { - properties.setProperty(entry.getKey().toString(), entry.getValue()); + return new SecorConfig(properties); } + }; - return new SecorConfig(properties); + public static SecorConfig load() throws ConfigurationException { + return mSecorConfig.get(); } - private SecorConfig(PropertiesConfiguration properties) { + /** + * Exposed for testability + * + * @param properties + */ + public SecorConfig(PropertiesConfiguration properties) { mProperties = properties; } @@ -69,18 +91,54 @@ public int getConsumerTimeoutMs() { return getInt("kafka.consumer.timeout.ms"); } + public String getConsumerAutoOffsetReset() { + return getString("kafka.consumer.auto.offset.reset"); + } + + public String getPartitionAssignmentStrategy() { + return getString("kafka.partition.assignment.strategy"); + } + public String getRebalanceMaxRetries() { return getString("kafka.rebalance.max.retries"); } + public String getRebalanceBackoffMs() { + return getString("kafka.rebalance.backoff.ms"); + } + public String getFetchMessageMaxBytes() { return getString("kafka.fetch.message.max.bytes"); } - public String getSocketReceieveBufferBytes() { + public String getSocketReceiveBufferBytes() { return getString("kafka.socket.receive.buffer.bytes"); } + public String getFetchMinBytes() { + return getString("kafka.fetch.min.bytes"); + } + + public String getFetchWaitMaxMs() { + return getString("kafka.fetch.wait.max.ms"); + } + + public String getDualCommitEnabled() { + return getString("kafka.dual.commit.enabled"); + } + + public String getOffsetsStorage() { + return getString("kafka.offsets.storage"); + } + + public boolean useKafkaTimestamp() { + return getBoolean("kafka.useTimestamp", false); + } + + public String getKafkaMessageTimestampClass() { + return getString("kafka.message.timestamp.className"); + } + public int getGeneration() { return getInt("secor.generation"); } @@ -97,6 +155,10 @@ public long getMaxFileAgeSeconds() { return getLong("secor.max.file.age.seconds"); } + public boolean getFileAgeYoungest() { + return getBoolean("secor.file.age.youngest"); + } + public long getOffsetsPerPartition() { return getLong("secor.offsets.per.partition"); } @@ -105,6 +167,20 @@ public int getMessagesPerSecond() { return getInt("secor.messages.per.second"); } + public String getS3FileSystem() { return getString("secor.s3.filesystem"); } + + public boolean getSeparateContainersForTopics() { + return getString("secor.swift.containers.for.each.topic").toLowerCase().equals("true"); + } + + public String getSwiftContainer() { + return getString("secor.swift.container"); + } + + public String getSwiftPath() { + return getString("secor.swift.path"); + } + public String getS3Bucket() { return getString("secor.s3.bucket"); } @@ -113,6 +189,18 @@ public String getS3Path() { return getString("secor.s3.path"); } + public String getS3AlternativePath() { + return getString("secor.s3.alternative.path"); + } + + public String getS3AlterPathDate() { + return getString("secor.s3.alter.path.date"); + } + + public String getS3Prefix() { + return getS3FileSystem() + "://" + getS3Bucket() + "/" + getS3Path(); + } + public String getLocalPath() { return getString("secor.local.path"); } @@ -121,6 +209,14 @@ public String getKafkaTopicFilter() { return getString("secor.kafka.topic_filter"); } + public String getKafkaTopicBlacklist() { + return getString("secor.kafka.topic_blacklist"); + } + + public String getKafkaTopicUploadAtMinuteMarkFilter() { return getString("secor.kafka.upload_at_minute_mark.topic_filter");} + + public int getUploadMinuteMark(){ return getInt("secor.upload.minute_mark");} + public String getKafkaGroup() { return getString("secor.kafka.group"); } @@ -137,6 +233,18 @@ public String getMessageParserClass() { return getString("secor.message.parser.class"); } + public String getUploaderClass() { + return getString("secor.upload.class", "com.pinterest.secor.uploader.Uploader"); + } + + public String getUploadManagerClass() { + return getString("secor.upload.manager.class"); + } + + public String getMessageTransformerClass(){ + return getString("secor.message.transformer.class"); + } + public int getTopicPartitionForgetSeconds() { return getInt("secor.topic_partition.forget.seconds"); } @@ -145,10 +253,18 @@ public int getLocalLogDeleteAgeHours() { return getInt("secor.local.log.delete.age.hours"); } + public String getFileExtension() { + return getString("secor.file.extension"); + } + public int getOstrichPort() { return getInt("ostrich.port"); } + public String getCloudService() { + return getString("cloud.service"); + } + public String getAwsAccessKey() { return getString("aws.access.key"); } @@ -157,6 +273,78 @@ public String getAwsSecretKey() { return getString("aws.secret.key"); } + public String getAwsEndpoint() { + return getString("aws.endpoint"); + } + + public String getAwsRole() { + return getString("aws.role"); + } + + public boolean getAwsClientPathStyleAccess() { + return getBoolean("aws.client.pathstyleaccess", false); + } + + public boolean getAwsProxyEnabled(){ + return getBoolean("aws.proxy.isEnabled"); + } + + public String getAwsProxyHttpHost() { + return getString("aws.proxy.http.host"); + } + + public int getAwsProxyHttpPort() { + return getInt("aws.proxy.http.port"); + } + + public String getAwsRegion() { + return getString("aws.region"); + } + + public String getAwsSseType() { + return getString("aws.sse.type"); + } + + public String getAwsSseKmsKey() { + return getString("aws.sse.kms.key"); + } + + public String getAwsSseCustomerKey() { + return getString("aws.sse.customer.key"); + } + + public String getSwiftTenant() { + return getString("swift.tenant"); + } + + public String getSwiftUsername() { + return getString("swift.username"); + } + + public String getSwiftPassword() { + return getString("swift.password"); + } + + public String getSwiftAuthUrl() { + return getString("swift.auth.url"); + } + + public String getSwiftPublic() { + return getString("swift.public"); + } + + public String getSwiftPort() { + return getString("swift.port"); + } + + public String getSwiftGetAuth() { + return getString("swift.use.get.auth"); + } + + public String getSwiftApiKey() { + return getString("swift.api.key"); + } + public String getQuboleApiToken() { return getString("qubole.api.token"); } @@ -165,44 +353,266 @@ public String getTsdbHostport() { return getString("tsdb.hostport"); } - public String getTsdbBlacklistTopics() { - return getString("tsdb.blacklist.topics"); + public String getStatsDHostPort() { + return getString("statsd.hostport"); + } + + public boolean getStatsDPrefixWithConsumerGroup(){ + return getBoolean("statsd.prefixWithConsumerGroup"); + } + + public String getMonitoringBlacklistTopics() { + return getString("monitoring.blacklist.topics"); + } + + public String getMonitoringPrefix() { + return getString("monitoring.prefix"); + } + + public long getMonitoringIntervalSeconds() { + return getLong("monitoring.interval.seconds"); } public String getMessageTimestampName() { return getString("message.timestamp.name"); } - + + public String getMessageTimestampNameSeparator() { + return getString("message.timestamp.name.separator"); + } + + public int getMessageTimestampId() { + return getInt("message.timestamp.id"); + } + + public String getMessageTimestampType() { + return getString("message.timestamp.type"); + } + public String getMessageTimestampInputPattern() { return getString("message.timestamp.input.pattern"); } + public boolean isMessageTimestampRequired() { + return mProperties.getBoolean("message.timestamp.required"); + } + + public String getMessageSplitFieldName() { + return getString("message.split.field.name"); + } + + public int getFinalizerLookbackPeriods() { + return getInt("secor.finalizer.lookback.periods", 10); + } + + public String getHivePrefix() { + return getString("secor.hive.prefix"); + } + + public String getHiveTableName(String topic) { + String key = "secor.hive.table.name." + topic; + return mProperties.getString(key, null); + } + + public boolean getQuboleEnabled() { + return getBoolean("secor.enable.qubole"); + } + + public long getQuboleTimeoutMs() { + return getLong("secor.qubole.timeout.ms"); + } + public String getCompressionCodec() { return getString("secor.compression.codec"); } - private void checkProperty(String name) { + public int getMaxMessageSizeBytes() { + return getInt("secor.max.message.size.bytes"); + } + + public String getFileReaderWriterFactory() { + return getString("secor.file.reader.writer.factory"); + } + + public String getFileReaderDelimiter(){ + String readerDelimiter = getString("secor.file.reader.Delimiter"); + if (readerDelimiter.length() > 1) { + throw new RuntimeException("secor.file.reader.Delimiter length can not be greater than 1 character"); + } + return readerDelimiter; + } + + public String getFileWriterDelimiter(){ + String writerDelimiter = getString("secor.file.writer.Delimiter"); + if (writerDelimiter.length() > 1) { + throw new RuntimeException("secor.file.writer.Delimiter length can not be greater than 1 character"); + } + return writerDelimiter; + } + + public String getPerfTestTopicPrefix() { + return getString("secor.kafka.perf_topic_prefix"); + } + + public String getZookeeperPath() { + return getString("secor.zookeeper.path"); + } + + public String getGsCredentialsPath() { + return getString("secor.gs.credentials.path"); + } + + public String getGsBucket() { + return getString("secor.gs.bucket"); + } + + public String getGsPath() { + return getString("secor.gs.path"); + } + + public int getGsConnectTimeoutInMs() { + return getInt("secor.gs.connect.timeout.ms", 3 * 60000); + } + + public int getGsReadTimeoutInMs() { + return getInt("secor.gs.read.timeout.ms", 3 * 60000); + } + + public boolean getGsDirectUpload() { + return getBoolean("secor.gs.upload.direct"); + } + + public int getFinalizerDelaySeconds() { + return getInt("partitioner.finalizer.delay.seconds"); + } + + public boolean getS3MD5HashPrefix() { + return getBoolean("secor.s3.prefix.md5hash"); + } + + public String getAzureEndpointsProtocol() { return getString("secor.azure.endpoints.protocol"); } + + public String getAzureAccountName() { return getString("secor.azure.account.name"); } + + public String getAzureAccountKey() { return getString("secor.azure.account.key"); } + + public String getAzureContainer() { return getString("secor.azure.container.name"); } + + public String getAzurePath() { return getString("secor.azure.path"); } + + public Map getProtobufMessageClassPerTopic() { + String prefix = "secor.protobuf.message.class"; + Iterator keys = mProperties.getKeys(prefix); + Map protobufClasses = new HashMap(); + while (keys.hasNext()) { + String key = keys.next(); + String className = mProperties.getString(key); + protobufClasses.put(key.substring(prefix.length() + 1), className); + } + return protobufClasses; + } + + public Map getThriftMessageClassPerTopic() { + String prefix = "secor.thrift.message.class"; + Iterator keys = mProperties.getKeys(prefix); + Map thriftClasses = new HashMap(); + while (keys.hasNext()) { + String key = keys.next(); + String className = mProperties.getString(key); + thriftClasses.put(key.substring(prefix.length() + 1), className); + } + return thriftClasses; + } + + public TimeZone getTimeZone() { + String timezone = getString("secor.parser.timezone"); + return Strings.isNullOrEmpty(timezone) ? TimeZone.getTimeZone("UTC") : TimeZone.getTimeZone(timezone); + } + + public boolean getBoolean(String name, boolean defaultValue) { + return mProperties.getBoolean(name, defaultValue); + } + + public boolean getBoolean(String name) { + return mProperties.getBoolean(name); + } + + public void checkProperty(String name) { if (!mProperties.containsKey(name)) { throw new RuntimeException("Failed to find required configuration option '" + name + "'."); } } - private String getString(String name) { + public String getString(String name) { checkProperty(name); return mProperties.getString(name); } - private int getInt(String name) { + public String getString(String name, String defaultValue) { + return mProperties.getString(name, defaultValue); + } + + public int getInt(String name) { checkProperty(name); return mProperties.getInt(name); } - private long getLong(String name) { + public int getInt(String name, int defaultValue) { + return mProperties.getInt(name, defaultValue); + } + + public long getLong(String name) { return mProperties.getLong(name); } - private String[] getStringArray(String name) { + public String[] getStringArray(String name) { return mProperties.getStringArray(name); } + + public String getThriftProtocolClass() { + return mProperties.getString("secor.thrift.protocol.class"); + } + + public String getMetricsCollectorClass() { + return getString("secor.monitoring.metrics.collector.class"); + } + + public Map getCustomTopicsNames() { + return getPropertyMapForPrefix("secor.kafka.customtopicname"); + } + + /** + * This method is used for fetching all the properties which start with the given prefix. + * It returns a Map of all those key-val. + * + * e.g. + * a.b.c=val1 + * a.b.d=val2 + * a.b.e=val3 + * + * If prefix is a.b then, + * These will be fetched as a map {c => val1, d => val2, e => val3} + * + * @param prefix + * @return + */ + public Map getPropertyMapForPrefix(String prefix) { + Iterator keys = mProperties.getKeys(prefix); + Map map = new HashMap(); + while (keys.hasNext()) { + String key = keys.next(); + String value = mProperties.getString(key); + map.put(key.substring(prefix.length() + 1), value); + } + return map; + } + + public Map getORCMessageSchema() { + return getPropertyMapForPrefix("secor.orc.message.schema"); + } + + public String getORCSchemaProviderClass(){ + return getString("secor.orc.schema.provider"); + } } diff --git a/src/main/java/com/pinterest/secor/common/TopicPartitionGroup.java b/src/main/java/com/pinterest/secor/common/TopicPartitionGroup.java new file mode 100644 index 000000000..8a9d3b434 --- /dev/null +++ b/src/main/java/com/pinterest/secor/common/TopicPartitionGroup.java @@ -0,0 +1,84 @@ +/** + * 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.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Topic partition group describes a kafka message topic-partitions pair. + * + * @author Henry Cai (hcai@pinterest.com) + */ +public class TopicPartitionGroup { + private String mTopic; + private int[] mPartitions; + + public TopicPartitionGroup(String topic, int[] partitions) { + mTopic = topic; + mPartitions = Arrays.copyOf(partitions, partitions.length); + } + + public TopicPartitionGroup(TopicPartition tp) { + this(tp.getTopic(), new int[]{tp.getPartition()}); + } + + public String getTopic() { + return mTopic; + } + + public int[] getPartitions() { + return mPartitions; + } + + public List getTopicPartitions() { + List tps = new ArrayList(); + for (int p : mPartitions) { + tps.add(new TopicPartition(mTopic, p)); + } + return tps; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + TopicPartitionGroup that = (TopicPartitionGroup) o; + + if (!Arrays.equals(mPartitions, that.mPartitions)) 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 + Arrays.hashCode(mPartitions); + return result; + } + + @Override + public String toString() { + return "TopicPartitionGroup{" + + "mTopic='" + mTopic + '\'' + + ", mPartitions=" + Arrays.toString(mPartitions) + + '}'; + } +} diff --git a/src/main/java/com/pinterest/secor/common/ZookeeperConnector.java b/src/main/java/com/pinterest/secor/common/ZookeeperConnector.java index 20c3e56d0..e86a31200 100644 --- a/src/main/java/com/pinterest/secor/common/ZookeeperConnector.java +++ b/src/main/java/com/pinterest/secor/common/ZookeeperConnector.java @@ -16,11 +16,14 @@ */ package com.pinterest.secor.common; +import com.google.common.base.Joiner; +import com.google.common.base.Strings; 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.commons.lang.StringUtils; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooDefs; @@ -44,6 +47,10 @@ public class ZookeeperConnector { private SecorConfig mConfig; private ZooKeeperClient mZookeeperClient; private HashMap mLocks; + private String mCommittedOffsetGroupPath; + + protected ZookeeperConnector() { + } public ZookeeperConnector(SecorConfig config) { mConfig = config; @@ -79,8 +86,18 @@ public void unlock(String lockPath) { mLocks.remove(lockPath); } - private String getCommittedOffsetGroupPath() { - return "/consumers/" + mConfig.getKafkaGroup() + "/offsets"; + protected String getCommittedOffsetGroupPath() { + if (Strings.isNullOrEmpty(mCommittedOffsetGroupPath)) { + String stripped = StringUtils.strip(mConfig.getKafkaZookeeperPath(), "/"); + mCommittedOffsetGroupPath = Joiner.on("/").skipNulls().join( + "", + stripped.equals("") ? null : stripped, + "consumers", + mConfig.getKafkaGroup(), + "offsets" + ); + } + return mCommittedOffsetGroupPath; } private String getCommittedOffsetTopicPath(String topic) { @@ -99,7 +116,7 @@ public long getCommittedOffsetCount(TopicPartition topicPartition) throws Except 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"); + LOG.warn("path {} does not exist in zookeeper", offsetPath); return -1; } } @@ -139,7 +156,7 @@ private void createMissingParents(String path) throws Exception { prefix += "/" + elements[i]; try { zookeeper.create(prefix, null, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - LOG.info("created path " + prefix); + LOG.info("created path {}", prefix); } catch (KeeperException.NodeExistsException exception) { } } @@ -149,11 +166,11 @@ 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); + 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); + LOG.info("setting zookeeper path {} value {}", offsetPath, count); // -1 matches any version zookeeper.setData(offsetPath, data, -1); } catch (KeeperException.NoNodeException exception) { @@ -167,7 +184,7 @@ public void deleteCommittedOffsetTopicCount(String topic) throws Exception { for (Integer partition : partitions) { TopicPartition topicPartition = new TopicPartition(topic, partition); String offsetPath = getCommittedOffsetPartitionPath(topicPartition); - LOG.info("deleting path " + offsetPath); + LOG.info("deleting path {}", offsetPath); zookeeper.delete(offsetPath, -1); } } @@ -176,7 +193,11 @@ public void deleteCommittedOffsetPartitionCount(TopicPartition topicPartition) throws Exception { String offsetPath = getCommittedOffsetPartitionPath(topicPartition); ZooKeeper zookeeper = mZookeeperClient.get(); - LOG.info("deleting path " + offsetPath); + LOG.info("deleting path {}", offsetPath); zookeeper.delete(offsetPath, -1); } + + protected void setConfig(SecorConfig config) { + this.mConfig = config; + } } diff --git a/src/main/java/com/pinterest/secor/consumer/Consumer.java b/src/main/java/com/pinterest/secor/consumer/Consumer.java index 73c3056f9..8ec3dbe4e 100644 --- a/src/main/java/com/pinterest/secor/consumer/Consumer.java +++ b/src/main/java/com/pinterest/secor/consumer/Consumer.java @@ -21,9 +21,12 @@ import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; import com.pinterest.secor.message.ParsedMessage; +import com.pinterest.secor.monitoring.MetricCollector; import com.pinterest.secor.parser.MessageParser; -import com.pinterest.secor.uploader.Uploader; import com.pinterest.secor.reader.MessageReader; +import com.pinterest.secor.transformer.MessageTransformer; +import com.pinterest.secor.uploader.UploadManager; +import com.pinterest.secor.uploader.Uploader; import com.pinterest.secor.util.ReflectionUtil; import com.pinterest.secor.writer.MessageWriter; import kafka.consumer.ConsumerTimeoutException; @@ -31,7 +34,6 @@ 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 @@ -46,27 +48,35 @@ public class Consumer extends Thread { private static final Logger LOG = LoggerFactory.getLogger(Consumer.class); - private SecorConfig mConfig; + protected SecorConfig mConfig; + protected MetricCollector mMetricCollector; - private MessageReader mMessageReader; - private MessageWriter mMessageWriter; - private MessageParser mMessageParser; - private Uploader mUploader; + protected MessageReader mMessageReader; + protected MessageWriter mMessageWriter; + protected MessageParser mMessageParser; + protected OffsetTracker mOffsetTracker; + protected MessageTransformer mMessageTransformer; + protected Uploader mUploader; // TODO(pawel): we should keep a count per topic partition. - private double mUnparsableMessages; + protected double mUnparsableMessages; 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); + mOffsetTracker = new OffsetTracker(); + mMessageReader = new MessageReader(mConfig, mOffsetTracker); + mMetricCollector = ReflectionUtil.createMetricCollector(mConfig.getMetricsCollectorClass()); + + FileRegistry fileRegistry = new FileRegistry(mConfig); + UploadManager uploadManager = ReflectionUtil.createUploadManager(mConfig.getUploadManagerClass(), mConfig); + + mUploader = ReflectionUtil.createUploader(mConfig.getUploaderClass()); + mUploader.init(mConfig, mOffsetTracker, fileRegistry, uploadManager, mMetricCollector); + mMessageWriter = new MessageWriter(mConfig, mOffsetTracker, fileRegistry); + mMessageParser = ReflectionUtil.createMessageParser(mConfig.getMessageParserClass(), mConfig); + mMessageTransformer = ReflectionUtil.createMessageTransformer(mConfig.getMessageTransformerClass(), mConfig); mUnparsableMessages = 0.; } @@ -79,49 +89,93 @@ public void run() { } catch (Exception e) { throw new RuntimeException("Failed to initialize the consumer", e); } + // check upload policy every N seconds or 10,000 messages/consumer timeouts + long checkEveryNSeconds = Math.min(10 * 60, mConfig.getMaxFileAgeSeconds() / 2); + long checkMessagesPerSecond = mConfig.getMessagesPerSecond(); + long nMessages = 0; + long lastChecked = System.currentTimeMillis(); while (true) { - Message rawMessage = null; + boolean hasMoreMessages = consumeNextMessage(); + if (!hasMoreMessages) { + break; + } + + long now = System.currentTimeMillis(); + if (nMessages++ % checkMessagesPerSecond == 0 || + (now - lastChecked) > checkEveryNSeconds * 1000) { + lastChecked = now; + checkUploadPolicy(); + } + } + checkUploadPolicy(); + } + + protected void checkUploadPolicy() { + try { + mUploader.applyPolicy(); + } catch (Exception e) { + throw new RuntimeException("Failed to apply upload policy", e); + } + } + + // @return whether there are more messages left to consume + protected boolean consumeNextMessage() { + Message rawMessage = null; + try { + boolean hasNext = mMessageReader.hasNext(); + if (!hasNext) { + return false; + } + 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. + LOG.trace("Consumer timed out", e); + } + if (rawMessage != null) { + // Before parsing, update the offset and remove any redundant data try { - boolean hasNext = mMessageReader.hasNext(); - if (!hasNext) { - return; + mMessageWriter.adjustOffset(rawMessage); + } catch (IOException e) { + throw new RuntimeException("Failed to adjust offset.", e); + } + ParsedMessage parsedMessage = null; + try { + Message transformedMessage = mMessageTransformer.transform(rawMessage); + parsedMessage = mMessageParser.parse(transformedMessage); + final double DECAY = 0.999; + mUnparsableMessages *= DECAY; + } catch (Throwable e) { + mMetricCollector.increment("consumer.message_errors.count", rawMessage.getTopic()); + + mUnparsableMessages++; + final double MAX_UNPARSABLE_MESSAGES = 1000.; + if (mUnparsableMessages > MAX_UNPARSABLE_MESSAGES) { + throw new RuntimeException("Failed to parse message " + rawMessage, e); } - 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. - LOG.trace("Consumer timed out", e); + LOG.warn("Failed to parse message {}", rawMessage, e); } - if (rawMessage != null) { - ParsedMessage parsedMessage = null; + + if (parsedMessage != null) { try { - parsedMessage = mMessageParser.parse(rawMessage); - final double DECAY = 0.999; - mUnparsableMessages *= DECAY; + mMessageWriter.write(parsedMessage); + + mMetricCollector.metric("consumer.message_size_bytes", rawMessage.getPayload().length, rawMessage.getTopic()); + mMetricCollector.increment("consumer.throughput_bytes", rawMessage.getPayload().length, rawMessage.getTopic()); } catch (Exception e) { - mUnparsableMessages++; - final double MAX_UNPARSABLE_MESSAGES = 1000.; - if (mUnparsableMessages > MAX_UNPARSABLE_MESSAGES) { - throw new RuntimeException("Failed to parse message " + rawMessage, e); - } - LOG.warn("Failed to parse message " + rawMessage, e); - continue; - } - if (parsedMessage != null) { - try { - mMessageWriter.write(parsedMessage); - } catch (IOException e) { - throw new RuntimeException("Failed to write message " + parsedMessage, 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); - } } + return true; + } + + /** + * Helper to get the offset tracker (used in tests) + * + * @return the offset tracker + */ + public OffsetTracker getOffsetTracker() { + return this.mOffsetTracker; } } diff --git a/src/main/java/com/pinterest/secor/io/FileReader.java b/src/main/java/com/pinterest/secor/io/FileReader.java new file mode 100644 index 000000000..959e7856a --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/FileReader.java @@ -0,0 +1,44 @@ +/** + * 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.io; + +import java.io.IOException; + +/** + * Generic file reader interface for a particular type of Secor output file + * + * Should be returned by a FileReaderWriterFactory that also knows how to build + * a corresponding FileReader (that is able to read the files written by this FileWriter). + * + * @author Praveen Murugesan (praveen@uber.com) + */ +public interface FileReader { + /** + * Get the next key/value from the file + * + * @return + * @throws IOException + */ + public KeyValue next() throws IOException; + + /** + * Close the file + * + * @throws IOException + */ + public void close() throws IOException; +} \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/io/FileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/FileReaderWriterFactory.java new file mode 100644 index 000000000..4b76b93cf --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/FileReaderWriterFactory.java @@ -0,0 +1,57 @@ +/** + * 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.io; + + +import com.pinterest.secor.common.LogFilePath; +import org.apache.hadoop.io.compress.CompressionCodec; + +import java.io.IOException; + +/** + * Provides a single factory class to make FileReader and FileWriter + * instances that can read from and write to the same type of output file. + * + * Implementers of this interface should provide a zero-argument constructor so that they can + * be constructed generically when referenced in configuration; see ReflectionUtil for details. + * + * @author Silas Davis (github-code@silasdavis.net) + */ +public interface FileReaderWriterFactory { + /** + * Build a FileReader instance to read from the target log file + * + * @param logFilePath the log file to read from + * @param codec the compression codec the file was written with (use null for no codec, + * or to auto-detect from file headers where supported) + * @return a FileReader instance to read from the target log file + * @throws IllegalAccessException + * @throws Exception + * @throws InstantiationException + */ + + public FileReader BuildFileReader(LogFilePath logFilePath, CompressionCodec codec) throws Exception; + /** + * Build a FileWriter instance to write to the target log file + * + * @param logFilePath the log file to read from + * @param codec the compression codec to write the file with + * @return a FileWriter instance to write to the target log file + * @throws Exception + */ + public FileWriter BuildFileWriter(LogFilePath logFilePath, CompressionCodec codec) throws Exception; +} \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/io/FileWriter.java b/src/main/java/com/pinterest/secor/io/FileWriter.java new file mode 100644 index 000000000..07d2a819f --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/FileWriter.java @@ -0,0 +1,52 @@ +/** + * 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.io; + +import java.io.IOException; + +/** + * Generic file writer interface for for a particular type of Secor output file + * + * Should be returned by a FileReaderWriterFactory that also know how to build + * a corresponding FileReader (that is able to read the files written by this FileWriter). + * + * @author Praveen Murugesan (praveen@uber.com) + */ +public interface FileWriter { + /** + * Get length of data written up to now to the underlying file + * + * @return + * @throws java.io.IOException + */ + public long getLength() throws IOException; + + /** + * Write the given key and value to the file + * + * @param keyValue + * @throws java.io.IOException + */ + public void write(KeyValue keyValue) throws IOException; + + /** + * Close the file + * + * @throws java.io.IOException + */ + public void close() throws IOException; +} \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/io/KeyValue.java b/src/main/java/com/pinterest/secor/io/KeyValue.java new file mode 100644 index 000000000..6dd60f3bd --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/KeyValue.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.io; + +/** + * Generic Object used to read next message from various file reader + * implementations + * + * @author Praveen Murugesan (praveen@uber.com) + * + */ +public class KeyValue { + + private final long mOffset; + private final byte[] mKafkaKey; + private final byte[] mValue; + private final long mTimestamp; + + // constructor + public KeyValue(long offset, byte[] value) { + this.mOffset = offset; + this.mKafkaKey = new byte[0]; + this.mValue = value; + this.mTimestamp = -1; + } + + // constructor + public KeyValue(long offset, byte[] kafkaKey, byte[] value) { + this.mOffset = offset; + this.mKafkaKey = kafkaKey; + this.mValue = value; + this.mTimestamp = -1; + } + + // constructor + public KeyValue(long offset, byte[] kafkaKey, byte[] value, long timestamp) { + this.mOffset = offset; + this.mKafkaKey = kafkaKey; + this.mValue = value; + this.mTimestamp = timestamp; + } + + public long getOffset() { + return this.mOffset; + } + + public byte[] getKafkaKey() { + return this.mKafkaKey; + } + + public byte[] getValue() { + return this.mValue; + } + + public long getTimestamp() { + return this.mTimestamp; + } + + public boolean hasKafkaKey() { + return this.mKafkaKey != null && this.mKafkaKey.length != 0; + } + + public boolean hasTimestamp(){ + return this.mTimestamp != -1; + } +} diff --git a/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java new file mode 100644 index 000000000..b4c23a032 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/DelimitedTextFileReaderWriterFactory.java @@ -0,0 +1,137 @@ +/** + * 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.io.impl; + +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.ByteArrayOutputStream; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; + +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileReaderWriterFactory; +import com.pinterest.secor.io.FileWriter; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.Compressor; +import org.apache.hadoop.io.compress.CodecPool; +import org.apache.hadoop.io.compress.Decompressor; + +import com.google.common.io.CountingOutputStream; +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.io.KeyValue; +import com.pinterest.secor.util.FileUtil; + +/** + * Delimited Text File Reader Writer with Compression + * + * @author Praveen Murugesan (praveen@uber.com) + */ +public class DelimitedTextFileReaderWriterFactory implements FileReaderWriterFactory { + private static final byte DELIMITER = '\n'; + + @Override + public FileReader BuildFileReader(LogFilePath logFilePath, CompressionCodec codec) + throws IllegalAccessException, IOException, InstantiationException { + return new DelimitedTextFileReader(logFilePath, codec); + } + + @Override + public FileWriter BuildFileWriter(LogFilePath logFilePath, CompressionCodec codec) throws IOException { + return new DelimitedTextFileWriter(logFilePath, codec); + } + + protected class DelimitedTextFileReader implements FileReader { + private final BufferedInputStream mReader; + private long mOffset; + private Decompressor mDecompressor = null; + + public DelimitedTextFileReader(LogFilePath path, CompressionCodec codec) throws IOException { + Path fsPath = new Path(path.getLogFilePath()); + FileSystem fs = FileUtil.getFileSystem(path.getLogFilePath()); + InputStream inputStream = fs.open(fsPath); + this.mReader = (codec == null) ? new BufferedInputStream(inputStream) + : new BufferedInputStream( + codec.createInputStream(inputStream, + mDecompressor = CodecPool.getDecompressor(codec))); + this.mOffset = path.getOffset(); + } + + @Override + public KeyValue next() throws IOException { + ByteArrayOutputStream messageBuffer = new ByteArrayOutputStream(); + int nextByte; + while ((nextByte = mReader.read()) != DELIMITER) { + if (nextByte == -1) { // end of stream? + if (messageBuffer.size() == 0) { // if no byte read + return null; + } else { // if bytes followed by end of stream: framing error + throw new EOFException( + "Non-empty message without delimiter"); + } + } + messageBuffer.write(nextByte); + } + return new KeyValue(this.mOffset++, messageBuffer.toByteArray()); + } + + @Override + public void close() throws IOException { + this.mReader.close(); + CodecPool.returnDecompressor(mDecompressor); + mDecompressor = null; + } + } + + protected class DelimitedTextFileWriter implements FileWriter { + private final CountingOutputStream mCountingStream; + private final BufferedOutputStream mWriter; + private Compressor mCompressor = null; + + public DelimitedTextFileWriter(LogFilePath path, CompressionCodec codec) throws IOException { + Path fsPath = new Path(path.getLogFilePath()); + FileSystem fs = FileUtil.getFileSystem(path.getLogFilePath()); + this.mCountingStream = new CountingOutputStream(fs.create(fsPath)); + this.mWriter = (codec == null) ? new BufferedOutputStream( + this.mCountingStream) : new BufferedOutputStream( + codec.createOutputStream(this.mCountingStream, + mCompressor = CodecPool.getCompressor(codec))); + } + + @Override + public long getLength() throws IOException { + assert this.mCountingStream != null; + this.mWriter.flush(); + return this.mCountingStream.getCount(); + } + + @Override + public void write(KeyValue keyValue) throws IOException { + this.mWriter.write(keyValue.getValue()); + this.mWriter.write(DELIMITER); + } + + @Override + public void close() throws IOException { + this.mWriter.close(); + CodecPool.returnCompressor(mCompressor); + mCompressor = null; + } + } +} diff --git a/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java new file mode 100644 index 000000000..d85f99233 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/FlexibleDelimitedFileReaderWriterFactory.java @@ -0,0 +1,172 @@ +/** +* 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.io.impl; + +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.ByteArrayOutputStream; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; + +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileReaderWriterFactory; +import com.pinterest.secor.io.FileWriter; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.Compressor; +import org.apache.hadoop.io.compress.CodecPool; +import org.apache.hadoop.io.compress.Decompressor; + +import org.apache.commons.configuration.ConfigurationException; + +import com.google.common.io.CountingOutputStream; +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.io.KeyValue; +import com.pinterest.secor.util.FileUtil; +import com.pinterest.secor.common.SecorConfig; + +/** +* Flexible Delimited Text File Reader Writer with Compression +* +* @author Ahsan Nabi Dar (ahsan@wego.com) +*/ +public class FlexibleDelimitedFileReaderWriterFactory implements FileReaderWriterFactory { + + @Override + public FileReader BuildFileReader(LogFilePath logFilePath, CompressionCodec codec) + throws IllegalAccessException, IOException, InstantiationException { + return new FlexibleDelimitedFileReader(logFilePath, codec); + } + + @Override + public FileWriter BuildFileWriter(LogFilePath logFilePath, CompressionCodec codec) throws IOException { + return new FlexibleDelimitedFileWriter(logFilePath, codec); + } + + + + protected class FlexibleDelimitedFileReader implements FileReader { + private final BufferedInputStream mReader; + private long mOffset; + private Decompressor mDecompressor = null; + private byte mDelimiter = getReaderDelimiter(); + + public FlexibleDelimitedFileReader(LogFilePath path, CompressionCodec codec) throws IOException { + Path fsPath = new Path(path.getLogFilePath()); + FileSystem fs = FileUtil.getFileSystem(path.getLogFilePath()); + InputStream inputStream = fs.open(fsPath); + this.mReader = (codec == null) ? new BufferedInputStream(inputStream) + : new BufferedInputStream( + codec.createInputStream(inputStream, + mDecompressor = CodecPool.getDecompressor(codec))); + this.mOffset = path.getOffset(); + } + + public byte getReaderDelimiter() { + byte delimiter = '\n'; + try { + String readerDelimiter = SecorConfig.load().getFileReaderDelimiter(); + if (!readerDelimiter.isEmpty()){ + delimiter = (byte)readerDelimiter.charAt(0); + } + } catch(ConfigurationException e) { + throw new RuntimeException("Error loading configuration from getFileReaderDelimiter()"); + } + return delimiter; + } + + @Override + public KeyValue next() throws IOException { + ByteArrayOutputStream messageBuffer = new ByteArrayOutputStream(); + int nextByte; + while ((nextByte = mReader.read()) != mDelimiter) { + if (nextByte == -1) { // end of stream? + if (messageBuffer.size() == 0) { // if no byte read + return null; + } else { // if bytes followed by end of stream: framing error + throw new EOFException( + "Non-empty message without delimiter"); + } + } + messageBuffer.write(nextByte); + } + return new KeyValue(this.mOffset++, messageBuffer.toByteArray()); + } + + @Override + public void close() throws IOException { + this.mReader.close(); + CodecPool.returnDecompressor(mDecompressor); + mDecompressor = null; + } + } + + protected class FlexibleDelimitedFileWriter implements FileWriter { + private final CountingOutputStream mCountingStream; + private final BufferedOutputStream mWriter; + private Compressor mCompressor = null; + private byte mDelimiter = getWriterDelimiter(); + private boolean addDelimiter = false; + + public FlexibleDelimitedFileWriter(LogFilePath path, CompressionCodec codec) throws IOException { + Path fsPath = new Path(path.getLogFilePath()); + FileSystem fs = FileUtil.getFileSystem(path.getLogFilePath()); + this.mCountingStream = new CountingOutputStream(fs.create(fsPath)); + this.mWriter = (codec == null) ? new BufferedOutputStream( + this.mCountingStream) : new BufferedOutputStream( + codec.createOutputStream(this.mCountingStream, + mCompressor = CodecPool.getCompressor(codec))); + } + + public byte getWriterDelimiter() { + byte delimiter = '\n'; + try { + String writerDelimiter = SecorConfig.load().getFileWriterDelimiter(); + if (!writerDelimiter.isEmpty()){ + addDelimiter = true; + delimiter = (byte)writerDelimiter.charAt(0); + } + } catch(ConfigurationException e) { + throw new RuntimeException("Error loading configuration from getFileWriterDelimiter()"); + } + return delimiter; + } + + @Override + public long getLength() throws IOException { + assert this.mCountingStream != null; + return this.mCountingStream.getCount(); + } + + @Override + public void write(KeyValue keyValue) throws IOException { + this.mWriter.write(keyValue.getValue()); + if (addDelimiter){ + this.mWriter.write(mDelimiter); + } + } + + @Override + public void close() throws IOException { + this.mWriter.close(); + CodecPool.returnCompressor(mCompressor); + mCompressor = null; + } + } +} diff --git a/src/main/java/com/pinterest/secor/io/impl/JsonORCFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/JsonORCFileReaderWriterFactory.java new file mode 100644 index 000000000..62359b7d1 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/JsonORCFileReaderWriterFactory.java @@ -0,0 +1,189 @@ +package com.pinterest.secor.io.impl; + +import java.io.IOException; +import java.io.StringWriter; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.Lz4Codec; +import org.apache.hadoop.io.compress.SnappyCodec; +import org.apache.orc.CompressionKind; +import org.apache.orc.OrcFile; +import org.apache.orc.Reader; +import org.apache.orc.RecordReader; +import org.apache.orc.TypeDescription; +import org.apache.orc.Writer; +import org.apache.orc.impl.ZlibCodec; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.gson.Gson; +import com.google.gson.JsonObject; +import com.pinterest.secor.common.FileRegistry; +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileReaderWriterFactory; +import com.pinterest.secor.io.FileWriter; +import com.pinterest.secor.io.KeyValue; +import com.pinterest.secor.util.ReflectionUtil; +import com.pinterest.secor.util.orc.JsonFieldFiller; +import com.pinterest.secor.util.orc.VectorColumnFiller; +import com.pinterest.secor.util.orc.VectorColumnFiller.JsonConverter; +import com.pinterest.secor.util.orc.schema.ORCScehmaProvider; + +/** + * ORC reader/writer implementation + * + * @author Ashish (ashu.impetus@gmail.com) + * + */ +public class JsonORCFileReaderWriterFactory implements FileReaderWriterFactory { + + private static final Logger LOG = LoggerFactory.getLogger(FileRegistry.class); + private ORCScehmaProvider schemaProvider; + + public JsonORCFileReaderWriterFactory(SecorConfig config) throws Exception { + schemaProvider = ReflectionUtil.createORCSchemaProvider( + config.getORCSchemaProviderClass(), config); + } + + @Override + public FileReader BuildFileReader(LogFilePath logFilePath, + CompressionCodec codec) throws Exception { + return new JsonORCFileReader(logFilePath, codec); + } + + @Override + public FileWriter BuildFileWriter(LogFilePath logFilePath, + CompressionCodec codec) throws Exception { + return new JsonORCFileWriter(logFilePath, codec); + } + + protected class JsonORCFileReader implements FileReader { + + private int rowIndex = 0; + private long offset; + private RecordReader rows; + private VectorizedRowBatch batch; + private TypeDescription schema; + + @SuppressWarnings("deprecation") + public JsonORCFileReader(LogFilePath logFilePath, CompressionCodec codec) + throws IOException { + schema = schemaProvider.getSchema(logFilePath.getTopic(), + logFilePath); + Path path = new Path(logFilePath.getLogFilePath()); + Reader reader = OrcFile.createReader(path, + OrcFile.readerOptions(new Configuration(true))); + offset = logFilePath.getOffset(); + rows = reader.rows(); + batch = reader.getSchema().createRowBatch(); + rows.nextBatch(batch); + } + + @Override + public KeyValue next() throws IOException { + boolean endOfBatch = false; + StringWriter sw = new StringWriter(); + + if (rowIndex > batch.size - 1) { + endOfBatch = !rows.nextBatch(batch); + rowIndex = 0; + } + + if (endOfBatch) { + rows.close(); + return null; + } + + try { + JsonFieldFiller.processRow(new JSONWriter(sw), batch, schema, + rowIndex); + } catch (JSONException e) { + LOG.error("Unable to parse json {}", sw.toString()); + return null; + } + rowIndex++; + return new KeyValue(offset++, sw.toString().getBytes("UTF-8")); + } + + @Override + public void close() throws IOException { + rows.close(); + } + } + + protected class JsonORCFileWriter implements FileWriter { + + private Gson gson = new Gson(); + private Writer writer; + private JsonConverter[] converters; + private VectorizedRowBatch batch; + private int rowIndex; + private TypeDescription schema; + + public JsonORCFileWriter(LogFilePath logFilePath, CompressionCodec codec) + throws IOException { + Configuration conf = new Configuration(); + Path path = new Path(logFilePath.getLogFilePath()); + schema = schemaProvider.getSchema(logFilePath.getTopic(), + logFilePath); + List fieldTypes = schema.getChildren(); + converters = new JsonConverter[fieldTypes.size()]; + for (int c = 0; c < converters.length; ++c) { + converters[c] = VectorColumnFiller.createConverter(fieldTypes + .get(c)); + } + + writer = OrcFile.createWriter(path, OrcFile.writerOptions(conf) + .compress(resolveCompression(codec)).setSchema(schema)); + batch = schema.createRowBatch(); + } + + @Override + public long getLength() throws IOException { + return writer.getRawDataSize(); + } + + @Override + public void write(KeyValue keyValue) throws IOException { + rowIndex = batch.size++; + VectorColumnFiller.fillRow(rowIndex, converters, schema, batch, + gson.fromJson(new String(keyValue.getValue()), + JsonObject.class)); + if (batch.size == batch.getMaxSize()) { + writer.addRowBatch(batch); + batch.reset(); + } + } + + @Override + public void close() throws IOException { + writer.addRowBatch(batch); + writer.close(); + } + } + + /** + * Used for returning the compression kind used in ORC + * + * @param codec + * @return + */ + private CompressionKind resolveCompression(CompressionCodec codec) { + if (codec instanceof Lz4Codec) + return CompressionKind.LZ4; + else if (codec instanceof SnappyCodec) + return CompressionKind.SNAPPY; + else if (codec instanceof ZlibCodec) + return CompressionKind.ZLIB; + else + return CompressionKind.NONE; + } +} diff --git a/src/main/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactory.java new file mode 100644 index 000000000..6a08eb421 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactory.java @@ -0,0 +1,191 @@ +/** + * 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.io.impl; + +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileReaderWriterFactory; +import com.pinterest.secor.io.FileWriter; +import com.pinterest.secor.io.KeyValue; +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.SequenceFile; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.msgpack.core.MessagePack; +import org.msgpack.core.MessagePacker; +import org.msgpack.core.MessageUnpacker; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Arrays; + +/** + * Sequence file reader writer implementation + * + * @author Praveen Murugesan (praveen@uber.com) + */ +public class MessagePackSequenceFileReaderWriterFactory implements FileReaderWriterFactory { + private static final int KAFKA_MESSAGE_OFFSET = 1; + private static final int KAFKA_HASH_KEY = 2; + private static final int KAFKA_MESSAGE_TIMESTAMP = 3; + private static final byte[] EMPTY_BYTES = new byte[0]; + + @Override + public FileReader BuildFileReader(LogFilePath logFilePath, CompressionCodec codec) throws Exception { + return new MessagePackSequenceFileReader(logFilePath); + } + + @Override + public FileWriter BuildFileWriter(LogFilePath logFilePath, CompressionCodec codec) throws IOException { + return new MessagePackSequenceFileWriter(logFilePath, codec); + } + + protected class MessagePackSequenceFileReader implements FileReader { + private final SequenceFile.Reader mReader; + private final BytesWritable mKey; + private final BytesWritable mValue; + + public MessagePackSequenceFileReader(LogFilePath path) throws Exception { + Configuration config = new Configuration(); + Path fsPath = new Path(path.getLogFilePath()); + FileSystem fs = FileUtil.getFileSystem(path.getLogFilePath()); + this.mReader = new SequenceFile.Reader(fs, fsPath, config); + this.mKey = (BytesWritable) mReader.getKeyClass().newInstance(); + this.mValue = (BytesWritable) mReader.getValueClass().newInstance(); + } + + @Override + public KeyValue next() throws IOException { + if (mReader.next(mKey, mValue)) { + MessageUnpacker unpacker = MessagePack.newDefaultUnpacker(mKey.getBytes()); + int mapSize = unpacker.unpackMapHeader(); + long offset = 0; + long timestamp = -1; + byte[] keyBytes = EMPTY_BYTES; + for (int i = 0; i < mapSize; i++) { + int key = unpacker.unpackInt(); + switch (key) { + case KAFKA_MESSAGE_OFFSET: + offset = unpacker.unpackLong(); + break; + case KAFKA_MESSAGE_TIMESTAMP: + timestamp = unpacker.unpackLong(); + break; + case KAFKA_HASH_KEY: + int keySize = unpacker.unpackBinaryHeader(); + keyBytes = new byte[keySize]; + unpacker.readPayload(keyBytes); + break; + } + } + unpacker.close(); + return new KeyValue(offset, keyBytes, Arrays.copyOfRange(mValue.getBytes(), 0, mValue.getLength()), timestamp); + } else { + return null; + } + } + + @Override + public void close() throws IOException { + this.mReader.close(); + } + } + + protected class MessagePackSequenceFileWriter implements FileWriter { + private final SequenceFile.Writer mWriter; + private final BytesWritable mKey; + private final BytesWritable mValue; + + public MessagePackSequenceFileWriter(LogFilePath path, CompressionCodec codec) throws IOException { + Configuration config = new Configuration(); + Path fsPath = new Path(path.getLogFilePath()); + FileSystem fs = FileUtil.getFileSystem(path.getLogFilePath()); + if (codec != null) { + this.mWriter = SequenceFile.createWriter(fs, config, fsPath, + BytesWritable.class, BytesWritable.class, + SequenceFile.CompressionType.BLOCK, codec); + } else { + this.mWriter = SequenceFile.createWriter(fs, config, fsPath, + BytesWritable.class, BytesWritable.class); + } + this.mKey = new BytesWritable(); + this.mValue = new BytesWritable(); + } + + @Override + public long getLength() throws IOException { + return this.mWriter.getLength(); + } + + @Override + public void write(KeyValue keyValue) throws IOException { + byte[] kafkaKey = keyValue.hasKafkaKey() ? keyValue.getKafkaKey() : new byte[0]; + long timestamp = keyValue.getTimestamp(); + final int timestampLength = (keyValue.hasTimestamp()) ? 10 : 0; + // output size estimate + // 1 - map header + // 1 - message pack key + // 9 - max kafka offset + // 1 - message pack key + // 9 - kafka timestamp + // 1 - message pack key + // 5 - max (sane) kafka key size + // N - size of kafka key + // = 27 + N + ByteArrayOutputStream out = new ByteArrayOutputStream(17 + timestampLength + kafkaKey.length); + MessagePacker packer = MessagePack.newDefaultPacker(out) + .packMapHeader(numberOfFieldsMappedInHeader(keyValue)) + .packInt(KAFKA_MESSAGE_OFFSET) + .packLong(keyValue.getOffset()); + + if (keyValue.hasTimestamp()) + packer.packInt(KAFKA_MESSAGE_TIMESTAMP) + .packLong(timestamp); + + if (keyValue.hasKafkaKey()) + packer.packInt(KAFKA_HASH_KEY) + .packBinaryHeader(kafkaKey.length) + .writePayload(kafkaKey); + + packer.close(); + byte[] outBytes = out.toByteArray(); + this.mKey.set(outBytes, 0, outBytes.length); + this.mValue.set(keyValue.getValue(), 0, keyValue.getValue().length); + this.mWriter.append(this.mKey, this.mValue); + } + + @Override + public void close() throws IOException { + this.mWriter.close(); + } + + private int numberOfFieldsMappedInHeader(KeyValue keyValue) { + int fields = 1; + + if (keyValue.hasKafkaKey()) + fields++; + + if (keyValue.hasTimestamp()) + fields++; + + return fields; + } + } +} diff --git a/src/main/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactory.java new file mode 100644 index 000000000..d3d3309a0 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactory.java @@ -0,0 +1,113 @@ +package com.pinterest.secor.io.impl; + +import java.io.IOException; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.proto.ProtoParquetReader; +import org.apache.parquet.proto.ProtoParquetWriter; + +import com.google.protobuf.Message; +import com.google.protobuf.Message.Builder; +import com.google.protobuf.MessageOrBuilder; +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileReaderWriterFactory; +import com.pinterest.secor.io.FileWriter; +import com.pinterest.secor.io.KeyValue; +import com.pinterest.secor.util.ParquetUtil; +import com.pinterest.secor.util.ProtobufUtil; + +/** + * Implementation for reading/writing protobuf messages to/from Parquet files. + * + * @author Michael Spector (spektom@gmail.com) + */ +public class ProtobufParquetFileReaderWriterFactory implements FileReaderWriterFactory { + + private ProtobufUtil protobufUtil; + + protected final int blockSize; + protected final int pageSize; + protected final boolean enableDictionary; + protected final boolean validating; + + public ProtobufParquetFileReaderWriterFactory(SecorConfig config) { + protobufUtil = new ProtobufUtil(config); + + blockSize = ParquetUtil.getParquetBlockSize(config); + pageSize = ParquetUtil.getParquetPageSize(config); + enableDictionary = ParquetUtil.getParquetEnableDictionary(config); + validating = ParquetUtil.getParquetValidation(config); + } + + @Override + public FileReader BuildFileReader(LogFilePath logFilePath, CompressionCodec codec) throws Exception { + return new ProtobufParquetFileReader(logFilePath, codec); + } + + @Override + public FileWriter BuildFileWriter(LogFilePath logFilePath, CompressionCodec codec) throws Exception { + return new ProtobufParquetFileWriter(logFilePath, codec); + } + + protected class ProtobufParquetFileReader implements FileReader { + + private ParquetReader reader; + private long offset; + + public ProtobufParquetFileReader(LogFilePath logFilePath, CompressionCodec codec) throws IOException { + Path path = new Path(logFilePath.getLogFilePath()); + reader = ProtoParquetReader.builder(path).build(); + offset = logFilePath.getOffset(); + } + + @Override + public KeyValue next() throws IOException { + Builder messageBuilder = (Builder) reader.read(); + if (messageBuilder != null) { + return new KeyValue(offset++, messageBuilder.build().toByteArray()); + } + return null; + } + + @Override + public void close() throws IOException { + reader.close(); + } + } + + protected class ProtobufParquetFileWriter implements FileWriter { + + private ProtoParquetWriter writer; + private String topic; + + public ProtobufParquetFileWriter(LogFilePath logFilePath, CompressionCodec codec) throws IOException { + Path path = new Path(logFilePath.getLogFilePath()); + CompressionCodecName codecName = CompressionCodecName + .fromCompressionCodec(codec != null ? codec.getClass() : null); + topic = logFilePath.getTopic(); + writer = new ProtoParquetWriter(path, protobufUtil.getMessageClass(topic), codecName, + blockSize, pageSize, enableDictionary, validating); + } + + @Override + public long getLength() throws IOException { + return writer.getDataSize(); + } + + @Override + public void write(KeyValue keyValue) throws IOException { + Message message = protobufUtil.decodeMessage(topic, keyValue.getValue()); + writer.write(message); + } + + @Override + public void close() throws IOException { + writer.close(); + } + } +} diff --git a/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactory.java new file mode 100644 index 000000000..88492d5c2 --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactory.java @@ -0,0 +1,128 @@ +/** + * 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.io.impl; + +import java.io.IOException; +import java.util.Arrays; + +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileReaderWriterFactory; +import com.pinterest.secor.io.FileWriter; +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.apache.hadoop.io.compress.CompressionCodec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.io.KeyValue; +import com.pinterest.secor.util.FileUtil; + +/** + * Sequence file reader writer implementation + * + * @author Praveen Murugesan (praveen@uber.com) + */ +public class SequenceFileReaderWriterFactory implements FileReaderWriterFactory { + + private static final Logger LOG = LoggerFactory.getLogger(SequenceFileReaderWriterFactory.class); + + @Override + public FileReader BuildFileReader(LogFilePath logFilePath, CompressionCodec codec) throws Exception { + return new SequenceFileReader(logFilePath); + } + + @Override + public FileWriter BuildFileWriter(LogFilePath logFilePath, CompressionCodec codec) throws IOException { + return new SequenceFileWriter(logFilePath, codec); + } + + protected class SequenceFileReader implements FileReader { + private final SequenceFile.Reader mReader; + private final LongWritable mKey; + private final BytesWritable mValue; + + public SequenceFileReader(LogFilePath path) throws Exception { + Configuration config = new Configuration(); + Path fsPath = new Path(path.getLogFilePath()); + FileSystem fs = FileUtil.getFileSystem(path.getLogFilePath()); + this.mReader = new SequenceFile.Reader(fs, fsPath, config); + this.mKey = (LongWritable) mReader.getKeyClass().newInstance(); + this.mValue = (BytesWritable) mReader.getValueClass().newInstance(); + } + + @Override + public KeyValue next() throws IOException { + if (mReader.next(mKey, mValue)) { + return new KeyValue(mKey.get(), Arrays.copyOfRange(mValue.getBytes(), 0, mValue.getLength())); + } else { + return null; + } + } + + @Override + public void close() throws IOException { + this.mReader.close(); + } + } + + protected class SequenceFileWriter implements FileWriter { + private final SequenceFile.Writer mWriter; + private final LongWritable mKey; + private final BytesWritable mValue; + private final Path fsPath; + + public SequenceFileWriter(LogFilePath path, CompressionCodec codec) throws IOException { + Configuration config = new Configuration(); + fsPath = new Path(path.getLogFilePath()); + FileSystem fs = FileUtil.getFileSystem(path.getLogFilePath()); + if (codec != null) { + this.mWriter = SequenceFile.createWriter(fs, config, fsPath, + LongWritable.class, BytesWritable.class, + SequenceFile.CompressionType.BLOCK, codec); + } else { + this.mWriter = SequenceFile.createWriter(fs, config, fsPath, + LongWritable.class, BytesWritable.class); + } + this.mKey = new LongWritable(); + this.mValue = new BytesWritable(); + LOG.info("Created sequence file writer: {}", fsPath); + } + + @Override + public long getLength() throws IOException { + return this.mWriter.getLength(); + } + + @Override + public void write(KeyValue keyValue) throws IOException { + this.mKey.set(keyValue.getOffset()); + this.mValue.set(keyValue.getValue(), 0, keyValue.getValue().length); + this.mWriter.append(this.mKey, this.mValue); + } + + @Override + public void close() throws IOException { + this.mWriter.close(); + LOG.info("Closing sequence file writer: {}", fsPath); + } + } +} \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java b/src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java new file mode 100644 index 000000000..2ff16024d --- /dev/null +++ b/src/main/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactory.java @@ -0,0 +1,132 @@ +package com.pinterest.secor.io.impl; + +import java.io.IOException; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.thrift.ThriftParquetReader; +import org.apache.parquet.thrift.ThriftParquetWriter; +import org.apache.thrift.TBase; +import org.apache.thrift.TException; + +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileReaderWriterFactory; +import com.pinterest.secor.io.FileWriter; +import com.pinterest.secor.io.KeyValue; +import com.pinterest.secor.util.ParquetUtil; +import com.pinterest.secor.util.ThriftUtil; + +/** + * Adapted from + * com.pinterest.secor.io.impl.ProtobufParquetFileReaderWriterFactory + * Implementation for reading/writing thrift messages to/from Parquet files. + * + + */ +public class ThriftParquetFileReaderWriterFactory implements FileReaderWriterFactory { + + private ThriftUtil thriftUtil; + + protected final int blockSize; + protected final int pageSize; + protected final boolean enableDictionary; + protected final boolean validating; + + public ThriftParquetFileReaderWriterFactory(SecorConfig config) { + thriftUtil = new ThriftUtil(config); + + blockSize = ParquetUtil.getParquetBlockSize(config); + pageSize = ParquetUtil.getParquetPageSize(config); + enableDictionary = ParquetUtil.getParquetEnableDictionary(config); + validating = ParquetUtil.getParquetValidation(config); + } + + @Override + public FileReader BuildFileReader(LogFilePath logFilePath, CompressionCodec codec) throws Exception { + return new ThriftParquetFileReader(logFilePath, codec); + } + + @Override + public FileWriter BuildFileWriter(LogFilePath logFilePath, CompressionCodec codec) throws Exception { + return new ThriftParquetFileWriter(logFilePath, codec); + } + + protected class ThriftParquetFileReader implements FileReader { + + private ParquetReader> reader; + private long offset; + + public ThriftParquetFileReader(LogFilePath logFilePath, CompressionCodec codec) throws IOException { + Path path = new Path(logFilePath.getLogFilePath()); + Class messageClass = thriftUtil.getMessageClass(logFilePath.getTopic()); + reader = ThriftParquetReader.build(path).withThriftClass((Class>) messageClass).build(); + offset = logFilePath.getOffset(); + } + + @SuppressWarnings("rawtypes") + @Override + public KeyValue next() throws IOException { + TBase msg = reader.read(); + + if (msg != null) { + try { + return new KeyValue(offset++, thriftUtil.encodeMessage(msg)); + } catch (TException e) { + throw new IOException("cannot write message", e); + } catch (InstantiationException e) { + throw new IOException("cannot write message", e); + } catch (IllegalAccessException e) { + throw new IOException("cannot write message", e); + } + } + return null; + } + + @Override + public void close() throws IOException { + reader.close(); + } + } + + protected class ThriftParquetFileWriter implements FileWriter { + + @SuppressWarnings("rawtypes") + private ThriftParquetWriter writer; + private String topic; + + @SuppressWarnings({ "rawtypes", "unchecked" }) + public ThriftParquetFileWriter(LogFilePath logFilePath, CompressionCodec codec) throws IOException { + Path path = new Path(logFilePath.getLogFilePath()); + CompressionCodecName codecName = CompressionCodecName.fromCompressionCodec(codec != null ? codec.getClass() : null); + topic = logFilePath.getTopic(); + writer = new ThriftParquetWriter(path, thriftUtil.getMessageClass(topic), codecName, + blockSize, pageSize, enableDictionary, validating); + } + + @Override + public long getLength() throws IOException { + return writer.getDataSize(); + } + + @SuppressWarnings("unchecked") + @Override + public void write(KeyValue keyValue) throws IOException { + Object message; + try { + message = thriftUtil.decodeMessage(topic, keyValue.getValue()); + writer.write(message); + } catch (Exception e) { + throw new IOException("cannot write message", e); + } + } + + @Override + public void close() throws IOException { + writer.close(); + } + } +} diff --git a/src/main/java/com/pinterest/secor/main/ConsumerMain.java b/src/main/java/com/pinterest/secor/main/ConsumerMain.java index ac751edfb..b328cefad 100644 --- a/src/main/java/com/pinterest/secor/main/ConsumerMain.java +++ b/src/main/java/com/pinterest/secor/main/ConsumerMain.java @@ -62,11 +62,11 @@ public static void main(String[] args) { RateLimitUtil.configure(config); Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler() { public void uncaughtException(Thread thread, Throwable exception) { - LOG.error("Thread " + thread + " failed", exception); + LOG.error("Thread {} failed", thread, exception); System.exit(1); } }; - LOG.info("starting " + config.getConsumerThreads() + " consumer threads"); + LOG.info("starting {} consumer threads", config.getConsumerThreads()); LinkedList consumers = new LinkedList(); for (int i = 0; i < config.getConsumerThreads(); ++i) { Consumer consumer = new Consumer(config); diff --git a/src/main/java/com/pinterest/secor/main/LogFileVerifierMain.java b/src/main/java/com/pinterest/secor/main/LogFileVerifierMain.java index ec1d7ec40..a7a77083d 100644 --- a/src/main/java/com/pinterest/secor/main/LogFileVerifierMain.java +++ b/src/main/java/com/pinterest/secor/main/LogFileVerifierMain.java @@ -49,11 +49,13 @@ private static CommandLine parseArgs(String[] args) throws ParseException { .create("t")); options.addOption(OptionBuilder.withLongOpt("start_offset") .withDescription("offset identifying the first set of files to check") + .hasArg() .withArgName("") .withType(Long.class) .create("s")); options.addOption(OptionBuilder.withLongOpt("end_offset") .withDescription("offset identifying the last set of files to check") + .hasArg() .withArgName("") .withType(Long.class) .create("e")); diff --git a/src/main/java/com/pinterest/secor/main/ProgressMonitorMain.java b/src/main/java/com/pinterest/secor/main/ProgressMonitorMain.java index 425a1ef3f..a2dfc15de 100644 --- a/src/main/java/com/pinterest/secor/main/ProgressMonitorMain.java +++ b/src/main/java/com/pinterest/secor/main/ProgressMonitorMain.java @@ -21,6 +21,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.Executors; + /** * Progress monitor main. * @@ -36,11 +40,33 @@ public class ProgressMonitorMain { private static final Logger LOG = LoggerFactory.getLogger(ProgressMonitorMain.class); + private static void loop(ProgressMonitor progressMonitor, long interval) { + final ProgressMonitor monitor = progressMonitor; + Runnable runner = new Runnable() { + public void run() { + try { + monitor.exportStats(); + } catch (Throwable t) { + LOG.error("Progress monitor failed", t); + } + } + }; + + ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1); + scheduler.scheduleAtFixedRate(runner, 0, interval, TimeUnit.SECONDS); + } + public static void main(String[] args) { try { SecorConfig config = SecorConfig.load(); ProgressMonitor progressMonitor = new ProgressMonitor(config); - progressMonitor.exportStats(); + + long interval = config.getMonitoringIntervalSeconds(); + if (interval > 0) { + loop(progressMonitor, interval); + } else { + 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 index 192dac091..25c3a86ee 100644 --- a/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java +++ b/src/main/java/com/pinterest/secor/main/TestLogMessageProducerMain.java @@ -57,6 +57,25 @@ private static CommandLine parseArgs(String[] args) throws ParseException { .withArgName("") .withType(Number.class) .create("p")); + options.addOption(OptionBuilder.withLongOpt("type") + .withDescription("type of producer - [json, binary]") + .hasArg() + .withArgName("") + .withType(String.class) + .create("type")); + options.addOption(OptionBuilder.withLongOpt("broker") + .withDescription("broker string, e.g. localhost:9092") + .hasArg() + .withArgName("") + .withType(String.class) + .create("broker")); + options.addOption(OptionBuilder.withLongOpt("timeshift") + .withDescription("message timestamp adjustment in seconds, it will be deducted" + + " from current time") + .hasArg() + .withArgName("") + .withType(Number.class) + .create("timeshift")); CommandLineParser parser = new GnuParser(); return parser.parse(options, args); @@ -68,8 +87,13 @@ public static void main(String[] args) { String topic = commandLine.getOptionValue("topic"); int messages = ((Number) commandLine.getParsedOptionValue("messages")).intValue(); int producers = ((Number) commandLine.getParsedOptionValue("producers")).intValue(); + String broker = commandLine.getOptionValue("broker"); + String type = commandLine.getOptionValue("type"); + Number timeshiftNumber = ((Number)commandLine.getParsedOptionValue("timeshift")); + int timeshift = timeshiftNumber == null ? 0 : timeshiftNumber.intValue(); for (int i = 0; i < producers; ++i) { - TestLogMessageProducer producer = new TestLogMessageProducer(topic, messages); + TestLogMessageProducer producer = new TestLogMessageProducer( + topic, messages, type, broker, timeshift); producer.start(); } } catch (Throwable t) { diff --git a/src/main/java/com/pinterest/secor/message/Message.java b/src/main/java/com/pinterest/secor/message/Message.java index c50ba9444..f355325ed 100644 --- a/src/main/java/com/pinterest/secor/message/Message.java +++ b/src/main/java/com/pinterest/secor/message/Message.java @@ -26,17 +26,23 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ public class Message { + + private static final byte[] EMPTY_BYTES = new byte[0]; + private String mTopic; private int mKafkaPartition; private long mOffset; + private byte[] mKafkaKey; private byte[] mPayload; + private long mTimestamp; protected String fieldsToString() { return "topic='" + mTopic + '\'' + ", kafkaPartition=" + mKafkaPartition + ", offset=" + mOffset + - ", payload=" + new String(mPayload); - + ", kafkaKey=" + new String(mKafkaKey) + + ", payload=" + new String(mPayload) + + ", timestamp=" + mTimestamp; } @Override @@ -44,15 +50,22 @@ public String toString() { return "Message{" + fieldsToString() + '}'; } - public Message(String topic, int kafkaPartition, long offset, byte[] payload) { + public Message(String topic, int kafkaPartition, long offset, byte[] kafkaKey, byte[] payload, long timestamp) { mTopic = topic; mKafkaPartition = kafkaPartition; mOffset = offset; + mKafkaKey = kafkaKey; + if (mKafkaKey == null) { + mKafkaKey = EMPTY_BYTES; + } mPayload = payload; + if (mPayload == null) { + mPayload = EMPTY_BYTES; + } + mTimestamp = timestamp; } public String getTopic() { - return mTopic; } @@ -64,10 +77,18 @@ public long getOffset() { return mOffset; } + public byte[] getKafkaKey() { + return mKafkaKey; + } + public byte[] getPayload() { return mPayload; } + public long getTimestamp() { + return mTimestamp; + } + 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 index 47bec2b4c..d3d0bf65e 100644 --- a/src/main/java/com/pinterest/secor/message/ParsedMessage.java +++ b/src/main/java/com/pinterest/secor/message/ParsedMessage.java @@ -34,9 +34,9 @@ public String toString() { Arrays.toString(mPartitions) + '}'; } - public ParsedMessage(String topic, int kafkaPartition, long offset, byte[] payload, - String[] mPartitions) { - super(topic, kafkaPartition, offset, payload); + public ParsedMessage(String topic, int kafkaPartition, long offset, byte[] kafkaKey, byte[] payload, + String[] mPartitions, long timestamp) { + super(topic, kafkaPartition, offset, kafkaKey, payload, timestamp); this.mPartitions = mPartitions; } diff --git a/src/main/java/com/pinterest/secor/monitoring/MetricCollector.java b/src/main/java/com/pinterest/secor/monitoring/MetricCollector.java new file mode 100644 index 000000000..879c2cd3c --- /dev/null +++ b/src/main/java/com/pinterest/secor/monitoring/MetricCollector.java @@ -0,0 +1,47 @@ +package com.pinterest.secor.monitoring; + +/** + * Component which may be used to post metrics. + * + * All methods should be non-blocking and do not throw exceptions. + */ +public interface MetricCollector { + /** + * Increments the specified counter by one. + * Convenience method equivalent to {@link #increment(String, int, String)}. + * + * @param label metric name + * @param topic a tag which describes which topic this data is collected for + */ + void increment(String label, String topic); + + /** + * Adjusts the specified counter by a given delta + * + * @param label metric name + * @param delta the amount to adjust the counter by + * @param topic a tag which describes which topic this data is collected for + */ + void increment(String label, int delta, String topic); + + /** + * Used to track the statistical distribution of a set of values. + *

+ * Metrics are collected by tracking the count, min, max, mean (average), and a simple bucket-based histogram of + * the distribution. This distribution can be used to determine median, 90th percentile, etc. + * + * @param label metric name + * @param value the value to be incorporated in the distribution + * @param topic a tag which describes which topic this data is collected for + */ + void metric(String label, double value, String topic); + + /** + * Records the latest fixed value for the specified named gauge. + * + * @param label gauge name + * @param value the new reading of the gauge + * @param topic a tag which describes which topic this data is collected for + */ + void gauge(String label, double value, String topic); +} diff --git a/src/main/java/com/pinterest/secor/monitoring/OstrichMetricCollector.java b/src/main/java/com/pinterest/secor/monitoring/OstrichMetricCollector.java new file mode 100644 index 000000000..c3cb97d32 --- /dev/null +++ b/src/main/java/com/pinterest/secor/monitoring/OstrichMetricCollector.java @@ -0,0 +1,25 @@ +package com.pinterest.secor.monitoring; + +import com.twitter.ostrich.stats.Stats; + +public class OstrichMetricCollector implements MetricCollector { + @Override + public void increment(String label, String topic) { + Stats.incr(label); + } + + @Override + public void increment(String label, int delta, String topic) { + Stats.incr(label, delta); + } + + @Override + public void metric(String label, double value, String topic) { + Stats.addMetric(label, (int) value); + } + + @Override + public void gauge(String label, double value, String topic) { + Stats.setGauge(label, value); + } +} diff --git a/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java b/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java new file mode 100644 index 000000000..1ea49b568 --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/DailyOffsetMessageParser.java @@ -0,0 +1,52 @@ +/** + * 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.commons.lang.StringUtils; + +import java.util.Date; + +/** + * Offset message parser groups messages based on the offset ranges. + * + * @author Ahsan Nabi Dar (ahsan@wego.com) + */ +public class DailyOffsetMessageParser extends TimestampedMessageParser { + + public DailyOffsetMessageParser(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[] dailyPartition = generatePartitions(new Date().getTime(), mUsingHourly, mUsingMinutely); + String dailyPartitionPath = StringUtils.join(dailyPartition, '/'); + String[] result = {dailyPartitionPath, offsetPrefix + partition}; + return result; + } + + @Override + public long extractTimestampMillis(final Message message) { + return new Date().getTime(); //Daily Timestamp generation + } + +} diff --git a/src/main/java/com/pinterest/secor/parser/DateMessageParser.java b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java index 3370b9352..0eb3606da 100644 --- a/src/main/java/com/pinterest/secor/parser/DateMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/DateMessageParser.java @@ -18,9 +18,7 @@ import java.text.SimpleDateFormat; import java.util.Date; - -import net.minidev.json.JSONObject; -import net.minidev.json.JSONValue; +import java.util.TimeZone; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -28,22 +26,40 @@ import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; +import net.minidev.json.JSONObject; +import net.minidev.json.JSONValue; + /** - * DateMessageParser extracts timestamp field (specified by 'message.timestamp.name') + * DateMessageParser extracts the timestamp field (specified by 'message.timestamp.name') * and the date pattern (specified by 'message.timestamp.input.pattern') - * + * * @see http://docs.oracle.com/javase/6/docs/api/java/text/SimpleDateFormat.html - * + * * @author Lucas Zago (lucaszago@gmail.com) - * + * */ public class DateMessageParser extends MessageParser { + private static final Logger LOG = LoggerFactory.getLogger(DateMessageParser.class); protected static final String defaultDate = "dt=1970-01-01"; - protected static final String defaultFormatter = "yyyy-MM-dd"; + protected SimpleDateFormat outputFormatter; + protected Object inputPattern; + protected SimpleDateFormat inputFormatter; + + protected final String mDtPrefix; public DateMessageParser(SecorConfig config) { super(config); + + TimeZone timeZone = config.getTimeZone(); + inputPattern = mConfig.getMessageTimestampInputPattern(); + inputFormatter = new SimpleDateFormat(inputPattern.toString()); + inputFormatter.setTimeZone(timeZone); + + outputFormatter = new SimpleDateFormat(TimestampedMessageParser.usingDateFormat(config)); + outputFormatter.setTimeZone(timeZone); + + mDtPrefix = TimestampedMessageParser.usingDatePrefix(config); } @Override @@ -52,24 +68,18 @@ public String[] extractPartitions(Message message) { String result[] = { defaultDate }; if (jsonObject != null) { - Object fieldValue = jsonObject.get(mConfig.getMessageTimestampName()); - Object inputPattern = mConfig.getMessageTimestampInputPattern(); + Object fieldValue = getJsonFieldValue(jsonObject); if (fieldValue != null && inputPattern != null) { try { - SimpleDateFormat inputFormatter = new SimpleDateFormat(inputPattern.toString()); - SimpleDateFormat outputFormatter = new SimpleDateFormat(defaultFormatter); Date dateFormat = inputFormatter.parse(fieldValue.toString()); - result[0] = "dt=" + outputFormatter.format(dateFormat); - return result; + result[0] = mDtPrefix + outputFormatter.format(dateFormat); } catch (Exception e) { - LOG.warn("Impossible to convert date = " + fieldValue.toString() - + " for the input pattern = " + inputPattern.toString() - + ". Using date default=" + result[0]); + LOG.warn("Impossible to convert date = {} with the input pattern = {}. Using date default = {}", + fieldValue.toString(), inputPattern.toString(), result[0]); } } } return result; } - } diff --git a/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java b/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java new file mode 100644 index 000000000..9a61e3105 --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/Iso8601MessageParser.java @@ -0,0 +1,65 @@ +/** + * 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 net.minidev.json.JSONObject; +import net.minidev.json.JSONValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.xml.bind.DatatypeConverter; +import java.util.Date; + +/** + * Iso8601MessageParser extracts timestamp field (specified by 'message.timestamp.name') + * + * @author Jurriaan Pruis (email@jurriaanpruis.nl) + * + */ +public class Iso8601MessageParser extends TimestampedMessageParser { + private final boolean m_timestampRequired; + + public Iso8601MessageParser(SecorConfig config) { + super(config); + m_timestampRequired = config.isMessageTimestampRequired(); + } + + @Override + public long extractTimestampMillis(final Message message) { + JSONObject jsonObject = (JSONObject) JSONValue.parse(message.getPayload()); + Object fieldValue = jsonObject != null ? getJsonFieldValue(jsonObject) : null; + + if (m_timestampRequired && fieldValue == null) { + throw new RuntimeException("Missing timestamp field for message: " + message); + } + + if (fieldValue != null) { + try { + Date dateFormat = DatatypeConverter.parseDateTime(fieldValue.toString()).getTime(); + return dateFormat.getTime(); + } catch (IllegalArgumentException ex) { + if (m_timestampRequired){ + throw new RuntimeException("Bad timestamp field for message: " + message); + } + } + } + + return 0; + } +} diff --git a/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java b/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java index e5eb91c94..0d09b7e34 100644 --- a/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/JsonMessageParser.java @@ -26,31 +26,24 @@ * from JSON data and partitions data by date. */ public class JsonMessageParser extends TimestampedMessageParser { + private final boolean m_timestampRequired; + public JsonMessageParser(SecorConfig config) { super(config); + m_timestampRequired = config.isMessageTimestampRequired(); } @Override - public long extractTimestampMillis(final Message message) throws ClassCastException { + public long extractTimestampMillis(final Message message) { JSONObject jsonObject = (JSONObject) JSONValue.parse(message.getPayload()); if (jsonObject != null) { - Object fieldValue = jsonObject.get(mConfig.getMessageTimestampName()); + Object fieldValue = getJsonFieldValue(jsonObject); if (fieldValue != null) { - long timestamp = 0; - if (fieldValue instanceof Number) { - timestamp = ((Number) fieldValue).longValue(); - } else { - // Sadly, I don't know of a better way to support all numeric types in Java - try { - timestamp = Long.valueOf(fieldValue.toString()); - } catch (NumberFormatException e) { - timestamp = Double.valueOf(fieldValue.toString()).longValue(); - } - } - return toMillis(timestamp); + return toMillis(Double.valueOf(fieldValue.toString()).longValue()); } + } else if (m_timestampRequired) { + throw new RuntimeException("Missing timestamp field for message: " + message); } return 0; } - } diff --git a/src/main/java/com/pinterest/secor/parser/MessagePackParser.java b/src/main/java/com/pinterest/secor/parser/MessagePackParser.java new file mode 100644 index 000000000..8baa4ffd4 --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/MessagePackParser.java @@ -0,0 +1,58 @@ +/** + * 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.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; +import org.msgpack.jackson.dataformat.MessagePackFactory; + +import java.util.HashMap; + +/** + * MessagePack timestamped message parser. + * Requires a second or ms timestamp. + * Does not support message.timestamp.input.pattern. + * + * @author Zack Dever (zack@rd.io) + */ +public class MessagePackParser extends TimestampedMessageParser { + private ObjectMapper mMessagePackObjectMapper; + private TypeReference mTypeReference; + + public MessagePackParser(SecorConfig config) { + super(config); + mMessagePackObjectMapper = new ObjectMapper(new MessagePackFactory()); + mTypeReference = new TypeReference>(){}; + } + + @Override + public long extractTimestampMillis(Message message) throws Exception { + HashMap msgHash = mMessagePackObjectMapper.readValue(message.getPayload(), + mTypeReference); + Object timestampValue = msgHash.get(mConfig.getMessageTimestampName()); + + if (timestampValue instanceof Number) { + return toMillis(((Number) timestampValue).longValue()); + } else if (timestampValue instanceof String) { + return toMillis(Long.parseLong((String) timestampValue)); + } else { + return toMillis((Long) timestampValue); + } + } +} diff --git a/src/main/java/com/pinterest/secor/parser/MessageParser.java b/src/main/java/com/pinterest/secor/parser/MessageParser.java index b0770760e..f21e4eeb3 100644 --- a/src/main/java/com/pinterest/secor/parser/MessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/MessageParser.java @@ -19,6 +19,11 @@ import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; import com.pinterest.secor.message.ParsedMessage; +import net.minidev.json.JSONObject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.regex.Pattern; // 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 @@ -31,16 +36,54 @@ */ public abstract class MessageParser { protected SecorConfig mConfig; + protected String[] mNestedFields; + protected final String offsetPrefix; + private static final Logger LOG = LoggerFactory.getLogger(MessageParser.class); public MessageParser(SecorConfig config) { mConfig = config; + offsetPrefix = usingOffsetPrefix(mConfig); + if (mConfig.getMessageTimestampName() != null && + !mConfig.getMessageTimestampName().isEmpty() && + mConfig.getMessageTimestampNameSeparator() != null && + !mConfig.getMessageTimestampNameSeparator().isEmpty()) { + String separatorPattern = Pattern.quote(mConfig.getMessageTimestampNameSeparator()); + mNestedFields = mConfig.getMessageTimestampName().split(separatorPattern); + } + } + + static String usingOffsetPrefix(SecorConfig config) { + return config.getString("secor.offsets.prefix"); } public ParsedMessage parse(Message message) throws Exception { String[] partitions = extractPartitions(message); return new ParsedMessage(message.getTopic(), message.getKafkaPartition(), - message.getOffset(), message.getPayload(), partitions); + message.getOffset(), message.getKafkaKey(), + message.getPayload(), partitions, message.getTimestamp()); } public abstract String[] extractPartitions(Message payload) throws Exception; + + public Object getJsonFieldValue(JSONObject jsonObject) { + Object fieldValue = null; + if (mNestedFields != null) { + Object finalValue = null; + for (int i=0; i < mNestedFields.length; i++) { + if (!jsonObject.containsKey(mNestedFields[i])) { + LOG.warn("Could not find key {} in message", mConfig.getMessageTimestampName()); + break; + } + if (i < (mNestedFields.length -1)) { + jsonObject = (JSONObject) jsonObject.get(mNestedFields[i]); + } else { + finalValue = jsonObject.get(mNestedFields[i]); + } + } + fieldValue = finalValue; + } else { + fieldValue = jsonObject.get(mConfig.getMessageTimestampName()); + } + return fieldValue; + } } diff --git a/src/main/java/com/pinterest/secor/parser/OffsetMessageParser.java b/src/main/java/com/pinterest/secor/parser/OffsetMessageParser.java index ddb1c69e6..78a6df1a9 100644 --- a/src/main/java/com/pinterest/secor/parser/OffsetMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/OffsetMessageParser.java @@ -34,7 +34,7 @@ public String[] extractPartitions(Message message) throws Exception { long offset = message.getOffset(); long offsetsPerPartition = mConfig.getOffsetsPerPartition(); long partition = (offset / offsetsPerPartition) * offsetsPerPartition; - String[] result = {"offset=" + partition}; + String[] result = {offsetPrefix + 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 index ec9d78f8a..3ab672895 100644 --- a/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java +++ b/src/main/java/com/pinterest/secor/parser/PartitionFinalizer.java @@ -18,19 +18,17 @@ import com.pinterest.secor.common.*; import com.pinterest.secor.message.Message; +import com.pinterest.secor.util.CompressionUtil; import com.pinterest.secor.util.FileUtil; import com.pinterest.secor.util.ReflectionUtil; import org.apache.hadoop.io.compress.CompressionCodec; -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; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Stack; /** * Partition finalizer writes _SUCCESS files to date partitions that very likely won't be receiving @@ -41,190 +39,172 @@ 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; - private String mFileExtension; + private final SecorConfig mConfig; + private final ZookeeperConnector mZookeeperConnector; + private final TimestampedMessageParser mMessageParser; + private final KafkaClient mKafkaClient; + private final QuboleClient mQuboleClient; + private final String mFileExtension; + private final int mLookbackPeriods; public PartitionFinalizer(SecorConfig config) throws Exception { mConfig = config; mKafkaClient = new KafkaClient(mConfig); mZookeeperConnector = new ZookeeperConnector(mConfig); - mThriftMessageParser = new ThriftMessageParser(mConfig); + mMessageParser = (TimestampedMessageParser) ReflectionUtil.createMessageParser( + mConfig.getMessageParserClass(), mConfig); mQuboleClient = new QuboleClient(mConfig); - if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { - CompressionCodec codec = (CompressionCodec) ReflectionUtil.createCompressionCodec(mConfig.getCompressionCodec()); + if (mConfig.getFileExtension() != null && !mConfig.getFileExtension().isEmpty()) { + mFileExtension = mConfig.getFileExtension(); + } else if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { + CompressionCodec codec = CompressionUtil.createCompressionCodec(mConfig.getCompressionCodec()); mFileExtension = codec.getDefaultExtension(); } else { mFileExtension = ""; } + mLookbackPeriods = config.getFinalizerLookbackPeriods(); + LOG.info("Lookback periods: " + mLookbackPeriods); } - private long getLastTimestampMillis(TopicPartition topicPartition) throws TException { - Message message = mKafkaClient.getLastMessage(topicPartition); - return mThriftMessageParser.extractTimestampMillis(message); - } - - private long getLastTimestampMillis(String topic) throws TException { + private String[] getFinalizedUptoPartitions(String topic) throws Exception { final int numPartitions = mKafkaClient.getNumPartitions(topic); - long max_timestamp = Long.MIN_VALUE; + List lastMessages = new ArrayList(numPartitions); + List committedMessages = new ArrayList(numPartitions); for (int partition = 0; partition < numPartitions; ++partition) { TopicPartition topicPartition = new TopicPartition(topic, partition); - long timestamp = getLastTimestampMillis(topicPartition); - if (timestamp > max_timestamp) { - max_timestamp = timestamp; + Message lastMessage = mKafkaClient.getLastMessage(topicPartition); + Message committedMessage = mKafkaClient.getCommittedMessage(topicPartition); + if (lastMessage == null || committedMessage == null) { + // This will happen if no messages have been posted to the given topic partition. + LOG.error("For topic {} partition {}, lastMessage: {}, committed: {}", + topicPartition.getTopic(), topicPartition.getPartition(), + lastMessage, committedMessage); + continue; } + lastMessages.add(lastMessage); + committedMessages.add(committedMessage); } - if (max_timestamp == Long.MIN_VALUE) { - return -1; - } - return max_timestamp; + return mMessageParser.getFinalizedUptoPartitions(lastMessages, committedMessages); } - 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 void finalizePartitionsUpTo(String topic, String[] uptoPartitions) throws Exception { + String prefix = FileUtil.getPrefix(topic, mConfig); + LOG.info("Finalize up to (but not include) {}, dim: {}", + uptoPartitions, uptoPartitions.length); - 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; - } + String[] previous = mMessageParser.getPreviousPartitions(uptoPartitions); + Stack toBeFinalized = new Stack(); + // Walk backwards to collect all partitions which are previous to the upTo partition + // Do not include the upTo partition + // Stop at the first partition which already have the SUCCESS file + for (int i = 0; i < mLookbackPeriods; i++) { + LOG.info("Looking for partition: " + Arrays.toString(previous)); + LogFilePath logFilePath = new LogFilePath(prefix, topic, previous, + mConfig.getGeneration(), 0, 0, mFileExtension); - 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, mFileExtension); - 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); + if (FileUtil.s3PathPrefixIsAltered(logFilePath.getLogFilePath(), mConfig)) { + logFilePath = logFilePath.withPrefix(FileUtil.getS3AlternativePrefix(mConfig)); } - } - 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, mFileExtension); String logFileDir = logFilePath.getLogFileDir(); - assert FileUtil.exists(logFileDir) : "FileUtil.exists(" + logFileDir + ")"; - String successFilePath = logFileDir + "/_SUCCESS"; - if (FileUtil.exists(successFilePath)) { - return; - } - try { - mQuboleClient.addPartition(topic, "dt='" + partitionStr + "'"); - } catch (Exception e) { - LOG.error("failed to finalize topic " + topic + " partition dt=" + partitionStr, - e); - continue; + if (FileUtil.exists(logFileDir)) { + String successFilePath = logFileDir + "/_SUCCESS"; + if (FileUtil.exists(successFilePath)) { + LOG.info( + "SuccessFile exist already, short circuit return. " + successFilePath); + break; + } + LOG.info("Folder {} exists and ready to be finalized.", logFileDir); + toBeFinalized.push(previous); + } else { + LOG.info("Folder {} doesn't exist, skip", logFileDir); } - LOG.info("touching file " + successFilePath); - FileUtil.touch(successFilePath); + previous = mMessageParser.getPreviousPartitions(previous); } - } - /** - * 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; + LOG.info("To be finalized partitions: {}", toBeFinalized); + if (toBeFinalized.isEmpty()) { + LOG.warn("There is no partitions to be finalized."); + return; } - 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; + // Now walk forward the collected partitions to do the finalization + // Note we are deliberately walking backwards and then forwards to make sure we don't + // end up in a situation that a later date partition is finalized and then the system + // crashes (which creates unfinalized partition folders in between) + while (!toBeFinalized.isEmpty()) { + String[] current = toBeFinalized.pop(); + LOG.info("Finalizing partition: " + Arrays.toString(current)); + // We only perform hive registration on the last dimension of the partition array + // i.e. only do hive registration for the hourly folder, but not for the daily + if (uptoPartitions.length == current.length) { + try { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < current.length; i++) { + String par = current[i]; + // We expect the partition array in the form of key=value if + // they need to go through hive registration + String[] parts = par.split("="); + assert parts.length == 2 : "wrong partition format: " + par; + if (i > 0) { + sb.append(","); + } + sb.append(parts[0]); + sb.append("='"); + sb.append(parts[1]); + sb.append("'"); + } + LOG.info("Hive partition string: " + sb); + + String hiveTableName = mConfig.getHiveTableName(topic); + LOG.info("Hive table name from config: {}", hiveTableName); + if (hiveTableName == null) { + String hivePrefix = null; + try { + hivePrefix = mConfig.getHivePrefix(); + hiveTableName = hivePrefix + topic; + LOG.info("Hive table name from prefix: {}", hiveTableName); + } catch (RuntimeException ex) { + LOG.warn("HivePrefix is not defined. Skip hive registration"); + } + } + if (hiveTableName != null && mConfig.getQuboleEnabled()) { + mQuboleClient.addPartition(hiveTableName, sb.toString()); + } + } catch (Exception e) { + LOG.error("failed to finalize topic " + topic, e); + continue; } } + + // Generate the SUCCESS file at the end + LogFilePath logFilePath = new LogFilePath(prefix, topic, current, + mConfig.getGeneration(), 0, 0, mFileExtension); + + if (FileUtil.s3PathPrefixIsAltered(logFilePath.getLogFilePath(), mConfig)) { + logFilePath = logFilePath.withPrefix(FileUtil.getS3AlternativePrefix(mConfig)); + LOG.info("Will finalize alternative s3 logFilePath {}", logFilePath); + } + + String logFileDir = logFilePath.getLogFileDir(); + String successFilePath = logFileDir + "/_SUCCESS"; + + LOG.info("touching file {}", successFilePath); + FileUtil.touch(successFilePath); } - if (minTimestamp == Long.MAX_VALUE) { - return -1; - } - return minTimestamp; } public void finalizePartitions() throws Exception { List topics = mZookeeperConnector.getCommittedOffsetTopics(); for (String topic : topics) { if (!topic.matches(mConfig.getKafkaTopicFilter())) { - LOG.info("skipping topic " + topic); + LOG.info("skipping topic {}", topic); } else { - 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); + LOG.info("finalizing topic {}", topic); + String[] partitions = getFinalizedUptoPartitions(topic); + LOG.info("finalized timestamp for topic {} is {}", topic , partitions); + if (partitions != null) { + finalizePartitionsUpTo(topic, partitions); } } } diff --git a/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java b/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java new file mode 100644 index 000000000..335203caf --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/PartitionedMessageParser.java @@ -0,0 +1,48 @@ +/** + * 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.commons.lang.StringUtils; + +import java.util.Date; + +/** + * Offset message parser groups messages based on the offset ranges. + * + * @author Ahsan Nabi Dar (ahsan@wego.com) + */ +public class PartitionedMessageParser extends TimestampedMessageParser { + + public PartitionedMessageParser(SecorConfig config) { + super(config); + } + + @Override + public String[] extractPartitions(Message message) throws Exception { + String[] dailyPartition = generatePartitions(new Date().getTime(), mUsingHourly, mUsingMinutely); + String dailyPartitionPath = StringUtils.join(dailyPartition, '/'); + String[] result = {dailyPartitionPath}; + return result; + } + + @Override + public long extractTimestampMillis(final Message message) { + return new Date().getTime(); //Daily Timestamp generation + } +} diff --git a/src/main/java/com/pinterest/secor/parser/Partitioner.java b/src/main/java/com/pinterest/secor/parser/Partitioner.java new file mode 100644 index 000000000..1153e946f --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/Partitioner.java @@ -0,0 +1,104 @@ +/** + * 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.message.Message; + +import java.util.List; + +/** + * The Partitioner knows when to finalize a file folder partition. + * + * A file folder partition (e.g. dt=2015-07-07) can be finalized when all + * messages in that date arrived. The caller (PartitionFinalizer) will do the + * finalization work (e.g. generate _SUCCESS file, perform hive registration) + * + * The partitioner provide the method to calculate the range of file + * folder partitions to be finalized and provide the method to iterate through + * the range. + * + * The caller will first provide a list of last-consumed messages for a given + * kafka topic and call #getFinalizedUptoPartitions to get the finalized-up-to + * partition and then walk backwards by calling #getPreviousPartitions to + * collect all the previous partitions which are ready to be finalized. + * + * Note that finalize-up-to partition itself is not inclusive in the range of + * partitions to be finalized. + * + * The caller might repeat this loop multiple times when the filesystem partition + * is multi-dimensional (e.g. [dt=2015-07-07,hr=05]). it will loop once for the + * hourly folder finalization and another time for the daily folder. + * + * Note that although we use daily/hourly partition illustrate the use of + * partitioner, it is be no means the partitioner can only work with timestamp + * based partitioning, it should also be able to work with offset based + * partitioning as long as we establish an iterating order within those + * partitions. + * + * @author Henry Cai (hcai@pinterest.com) + */ +public interface Partitioner { + /** + * Calculates the partition to finalize-up-to from a list of last-consumed + * messages and a list of last-enqueued messages. + * + * For each kafka topic/partition for a given topic, the caller will provide + * two messages: + * * lastMessage: the last message at the tail of the kafka queue + * * committedMessage: the message secor consumed and committed + * And then iterate over all the kafka topic partitions for the given topic, + * the caller will gather the above two messages into two lists. + * + * The Partitioner will compare the messages from all kafka partitions to + * see which one is the earliest to finalize up to. The partitioner will + * normally use the timestamp from the committedMessage to decide + * the finalize time. But for some slow topics where there is no new + * messages coming for a while (i.e. lastMessage == committedMessage), + * the partitioner can use the current time as the finalize time. + * + * Note that the up-to partition itself is not inclusive in the range to be + * finalized. For example, when the last message is in 2015-07-07, + * 7/7 itself is not complete yet. + * + * Note also that the partitioner might want to adjust down the finalize + * time to allow a safety lag for late arrival messages. e.g. adding one + * extra hour lag + * + * @param lastMessages the last message at the tail of the queue + * @param committedMessages the message secor consumed and committed + * + * @return a String array to represent a file folder partition to finalize up to + */ + String[] getFinalizedUptoPartitions(List lastMessages, + List committedMessages) throws Exception; + + /** + * Get the previous partition out of the incoming partition. + * E.g. for ["dt=2015-07-07","hr=05"], it will return ["dt=2015-07-07","hr=04"] + * + * Note that the implementation might return the previous sequence in daily/mixed forms, e.g. + * [dt=2015-07-07, hr=01] + * [dt=2015-07-07, hr=00] + * [dt=2015-07-07] <-- dt folder in between + * [dt=2015-07-06, hr=23] + * [dt=2015-07-07, hr=22] + * + * @param partition + * @return + */ + String[] getPreviousPartitions(String[] partition) throws Exception; +} diff --git a/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java new file mode 100644 index 000000000..3ae8b9a16 --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/ProtobufMessageParser.java @@ -0,0 +1,100 @@ +/** + * 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 java.io.IOException; +import java.util.regex.Pattern; + +import com.google.protobuf.Descriptors; +import com.google.protobuf.util.Timestamps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.protobuf.CodedInputStream; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; +import com.pinterest.secor.util.ProtobufUtil; + +/** + * Protocol buffer message timestamp extractor + * + * If secor.protobuf.message.class is not set assumes that the very + * first uint64 field in a message is the timestamp. Otherwise, + * uses message.timestamp.name as a path to get to the timestamp + * field within protobuf message. + * + * @author Liam Stewart (liam.stewart@gmail.com) + */ +public class ProtobufMessageParser extends TimestampedMessageParser { + + private static final Logger LOG = LoggerFactory.getLogger(ProtobufMessageParser.class); + + private ProtobufUtil protobufUtil; + private String[] timestampFieldPath; + + public ProtobufMessageParser(SecorConfig config) { + super(config); + + protobufUtil = new ProtobufUtil(config); + if (protobufUtil.isConfigured()) { + String timestampFieldName = mConfig.getMessageTimestampName(); + String timestampFieldSeparator = mConfig.getMessageTimestampNameSeparator(); + if (timestampFieldSeparator == null || timestampFieldSeparator.isEmpty()) { + timestampFieldSeparator = "."; + } + LOG.info("Using protobuf timestamp field path: {} with separator: {}", timestampFieldName, + timestampFieldSeparator); + timestampFieldPath = timestampFieldName.split(Pattern.quote(timestampFieldSeparator)); + } else { + LOG.info( + "Protobuf message class is not configured, will assume that timestamp is the first uint64 field"); + } + } + + @Override + public long extractTimestampMillis(final Message message) throws IOException { + return extractTimestampMillis(message.getTopic(), message.getPayload()); + } + + public long extractTimestampMillis(String topic, final byte[] bytes) throws IOException { + if (timestampFieldPath != null) { + com.google.protobuf.Message decodedMessage = protobufUtil.decodeMessage(topic, + bytes); + int i = 0; + for (; i < timestampFieldPath.length - 1; ++i) { + decodedMessage = (com.google.protobuf.Message) decodedMessage + .getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i])); + } + Object timestampObject = decodedMessage + .getField(decodedMessage.getDescriptorForType().findFieldByName(timestampFieldPath[i])); + if (timestampObject instanceof com.google.protobuf.Timestamp){ + return Timestamps.toMillis((com.google.protobuf.Timestamp) timestampObject); + }else { + return toMillis((Long) timestampObject); + } + } else { + // Assume that the timestamp field is the first field, is required, + // and is a uint64. + + CodedInputStream input = CodedInputStream.newInstance(bytes); + // Don't really care about the tag, but need to read it to get, to + // the payload. + input.readTag(); + return toMillis(input.readUInt64()); + } + } +} diff --git a/src/main/java/com/pinterest/secor/parser/QuboleClient.java b/src/main/java/com/pinterest/secor/parser/QuboleClient.java index e39228671..748ebc7d5 100644 --- a/src/main/java/com/pinterest/secor/parser/QuboleClient.java +++ b/src/main/java/com/pinterest/secor/parser/QuboleClient.java @@ -31,10 +31,13 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ public class QuboleClient { + private final long MAX_QUBOLE_WAIT_TIME_MS; + private String mApiToken; public QuboleClient(SecorConfig config) { mApiToken = config.getQuboleApiToken(); + MAX_QUBOLE_WAIT_TIME_MS = config.getQuboleTimeoutMs(); } private Map makeRequest(URL url, String body) throws IOException { @@ -66,7 +69,12 @@ private Map makeRequest(URL url, String body) throws IOException { // Get Response. InputStream inputStream = connection.getInputStream(); BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream)); - Map response = (Map) JSONValue.parse(reader); + Object responseObj = JSONValue.parse(reader); + if (!(responseObj instanceof Map)) { + throw new RuntimeException("command " + url + " body " + body + " unexpected " + + responseObj); + } + Map response = (Map)responseObj; if (response.get("status").equals("error")) { throw new RuntimeException("command " + url + " with body " + body + " failed " + JSONObject.toJSONString(response)); @@ -81,7 +89,7 @@ private Map makeRequest(URL url, String body) throws IOException { } private int query(String query) throws IOException { - URL url = new URL("http://api.qubole.com/api/v1.2/commands"); + URL url = new URL("https://api.qubole.com/api/v1.2/commands"); JSONObject queryJson = new JSONObject(); queryJson.put("query", query); String body = queryJson.toString(); @@ -89,9 +97,11 @@ private int query(String query) throws IOException { 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) { + private void waitForCompletion(int commandId, long timeout) throws IOException, InterruptedException { + URL url = new URL("https://api.qubole.com/api/v1.2/commands/" + commandId); + long endTime = System.currentTimeMillis() + timeout; + + while (System.currentTimeMillis() < endTime) { Map response = makeRequest(url, null); if (response.get("status").equals("done")) { return; @@ -100,13 +110,15 @@ private void waitForCompletion(int commandId) throws IOException, InterruptedExc ". Current status " + response.get("status")); Thread.sleep(3000); } + + throw new IOException("Qubole commandId" + commandId + " failed to return within timeout."); } 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); + waitForCompletion(commandId, MAX_QUBOLE_WAIT_TIME_MS); } } diff --git a/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java b/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java new file mode 100644 index 000000000..04fc482ee --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/RegexMessageParser.java @@ -0,0 +1,57 @@ +/** + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * RegexMessageParser extracts timestamp field (specified by 'message.timestamp.input.pattern') + * The pattern specifies the regular exp to extract the timestamp field from a free-text line. + * + * * @author Henry Cai (hcai@pinterest.com) + */ +public class RegexMessageParser extends TimestampedMessageParser { + private static final Logger LOG = LoggerFactory.getLogger(RegexMessageParser.class); + + private final Pattern mTsPattern; + + public RegexMessageParser(SecorConfig config) { + super(config); + String patStr = config.getMessageTimestampInputPattern(); + LOG.info("timestamp pattern: {}", patStr); + mTsPattern = Pattern.compile(patStr, Pattern.UNIX_LINES); + } + + @Override + public long extractTimestampMillis(final Message message) { + String line = new String(message.getPayload()); + Matcher m = mTsPattern.matcher(line); + if (m.find()) { + String tsValue = m.group(1); + if (tsValue != null) { + return toMillis(Long.parseLong(tsValue)); + } + } + throw new NumberFormatException("Cannot find timestamp field in: " + line); + } +} diff --git a/src/main/java/com/pinterest/secor/parser/SplitByFieldMessageParser.java b/src/main/java/com/pinterest/secor/parser/SplitByFieldMessageParser.java new file mode 100644 index 000000000..9b76f84d2 --- /dev/null +++ b/src/main/java/com/pinterest/secor/parser/SplitByFieldMessageParser.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.parser; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; +import net.minidev.json.JSONObject; +import net.minidev.json.JSONValue; +import org.apache.commons.lang3.ArrayUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +/** + * SplitByFieldMessageParser extracts event type field (specified by 'message.split.field.name') + * and timestamp field (specified by 'message.timestamp.name') + * from JSON data and splits data into multiple outputs by event type and then partitions each output by date. + * + * Caution: this parser doesn't support finalization of partitions. + */ +public class SplitByFieldMessageParser extends TimestampedMessageParser implements Partitioner { + private static final Logger LOG = LoggerFactory.getLogger(SplitByFieldMessageParser.class); + private final String mSplitFieldName; + + public SplitByFieldMessageParser(SecorConfig config) { + super(config); + + mSplitFieldName = config.getMessageSplitFieldName(); + } + + @Override + public long extractTimestampMillis(Message message) throws Exception { + throw new UnsupportedOperationException("Unsupported, use extractPartitions method instead"); + } + + @Override + public String[] extractPartitions(Message message) throws Exception { + JSONObject jsonObject = (JSONObject) JSONValue.parse(message.getPayload()); + if (jsonObject == null) { + throw new RuntimeException("Failed to parse message as Json object"); + } + + String eventType = extractEventType(jsonObject); + long timestampMillis = extractTimestampMillis(jsonObject); + + String[] timestampPartitions = generatePartitions(timestampMillis, mUsingHourly, mUsingMinutely); + return ArrayUtils.addAll(new String[]{eventType}, timestampPartitions); + } + + @Override + public String[] getFinalizedUptoPartitions(List lastMessages, + List committedMessages) throws Exception { + throw new UnsupportedOperationException("Partition finalization is not supported"); + } + + @Override + public String[] getPreviousPartitions(String[] partitions) throws Exception { + throw new UnsupportedOperationException("Partition finalization is not supported"); + } + + protected String extractEventType(JSONObject jsonObject) { + if (!jsonObject.containsKey(mSplitFieldName)) { + throw new RuntimeException("Could not find key " + mSplitFieldName + " in Json message"); + } + return jsonObject.get(mSplitFieldName).toString(); + } + + protected long extractTimestampMillis(JSONObject jsonObject) { + Object fieldValue = getJsonFieldValue(jsonObject); + if (fieldValue != null) { + return toMillis(Double.valueOf(fieldValue.toString()).longValue()); + } else { + throw new RuntimeException("Failed to extract timestamp from the message"); + } + } +} diff --git a/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java b/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java index d634baf44..f3349c9b1 100644 --- a/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/ThriftMessageParser.java @@ -5,9 +5,9 @@ * 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 - * + *

+ * 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. @@ -16,11 +16,15 @@ */ package com.pinterest.secor.parser; -import com.pinterest.secor.common.SecorConfig; -import com.pinterest.secor.message.Message; +import org.apache.commons.lang3.StringUtils; import org.apache.thrift.TDeserializer; import org.apache.thrift.TException; import org.apache.thrift.TFieldIdEnum; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TProtocolFactory; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; /** * Thrift message parser extracts date partitions from thrift messages. @@ -28,33 +32,57 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ public class ThriftMessageParser extends TimestampedMessageParser { - private TDeserializer mDeserializer; + private final TDeserializer mDeserializer; + private final ThriftPath mThriftPath; + private final String mTimestampType; + + class ThriftPath implements TFieldIdEnum { + private final String mFieldName; + private final short mFieldId; - public ThriftMessageParser(SecorConfig config) { + public ThriftPath(final String fieldName, final short fieldId) { + this.mFieldName = fieldName; + this.mFieldId = fieldId; + } + + @Override + public short getThriftFieldId() { + return mFieldId; + } + + @Override + public String getFieldName() { + return mFieldName; + } + } + + public ThriftMessageParser(SecorConfig config) + throws InstantiationException, IllegalAccessException, + ClassNotFoundException { super(config); - mDeserializer = new TDeserializer(); + TProtocolFactory protocolFactory = null; + String protocolName = mConfig.getThriftProtocolClass(); + + if (StringUtils.isNotEmpty(protocolName)) { + String factoryClassName = protocolName.concat("$Factory"); + protocolFactory = ((Class) Class.forName(factoryClassName)).newInstance(); + } else + protocolFactory = new TBinaryProtocol.Factory(); + + mDeserializer = new TDeserializer(protocolFactory); + mThriftPath = new ThriftPath(mConfig.getMessageTimestampName(),(short) mConfig.getMessageTimestampId()); + mTimestampType = mConfig.getMessageTimestampType(); } @Override public long extractTimestampMillis(final Message message) throws TException { - class ThriftTemplate implements TFieldIdEnum { - private final String mFieldName; - public ThriftTemplate(final String fieldName) { - this.mFieldName = fieldName; - } - - @Override - public short getThriftFieldId() { - return 1; - } - - @Override - public String getFieldName() { - return mFieldName; - } + long timestamp; + if ("i32".equals(mTimestampType)) { + timestamp = (long) mDeserializer.partialDeserializeI32(message.getPayload(), mThriftPath); + } else { + timestamp = mDeserializer.partialDeserializeI64(message.getPayload(), mThriftPath); } - long timestamp = mDeserializer.partialDeserializeI64(message.getPayload(), - new ThriftTemplate(mConfig.getMessageTimestampName())); + return toMillis(timestamp); } } diff --git a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java index 247a29943..ec6593ae3 100644 --- a/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java +++ b/src/main/java/com/pinterest/secor/parser/TimestampedMessageParser.java @@ -5,9 +5,9 @@ * 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 - * + *

+ * 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. @@ -18,24 +18,127 @@ import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.text.SimpleDateFormat; import java.util.Date; -import java.util.TimeZone; +import java.util.List; + +public abstract class TimestampedMessageParser extends MessageParser implements Partitioner { + + private static final Logger LOG = LoggerFactory.getLogger(TimestampedMessageParser.class); + + private static final long HOUR_IN_MILLIS = 3600L * 1000L; + private static final long DAY_IN_MILLIS = 3600L * 24 * 1000L; + private static final long MINUTE_IN_MILLIS = 60L * 1000L; + + /* + * IMPORTANT + * SimpleDateFormat are NOT thread-safe. + * Each parser needs to have their own local SimpleDateFormat or it'll cause race condition. + */ + protected final SimpleDateFormat mDtFormatter; + protected final SimpleDateFormat mHrFormatter; + protected final SimpleDateFormat mDtHrFormatter; + protected final int mFinalizerDelaySeconds; + protected final SimpleDateFormat mDtHrMinFormatter; + protected final SimpleDateFormat mMinFormatter; + + protected final String mDtFormat; + protected final String mHrFormat; + protected final String mMinFormat; + + protected final String mDtPrefix; + protected final String mHrPrefix; + protected final String mMinPrefix; + + protected final boolean mUsingHourly; + protected final boolean mUsingMinutely; + + protected final boolean mUseKafkaTimestamp; + -public abstract class TimestampedMessageParser extends MessageParser { public TimestampedMessageParser(SecorConfig config) { super(config); + + mUsingHourly = usingHourly(config); + mUsingMinutely = usingMinutely(config); + mUseKafkaTimestamp = useKafkaTimestamp(config); + mDtFormat = usingDateFormat(config); + mHrFormat = usingHourFormat(config); + mMinFormat = usingMinuteFormat(config); + + mDtPrefix = usingDatePrefix(config); + mHrPrefix = usingHourPrefix(config); + mMinPrefix = usingMinutePrefix(config); + + LOG.info("UsingHourly: {}", mUsingHourly); + LOG.info("UsingMin: {}", mUsingMinutely); + mFinalizerDelaySeconds = config.getFinalizerDelaySeconds(); + LOG.info("FinalizerDelaySeconds: {}", mFinalizerDelaySeconds); + + mDtFormatter = new SimpleDateFormat(mDtFormat); + mDtFormatter.setTimeZone(config.getTimeZone()); + + mHrFormatter = new SimpleDateFormat(mHrFormat); + mHrFormatter.setTimeZone(config.getTimeZone()); + + mMinFormatter = new SimpleDateFormat(mMinFormat); + mMinFormatter.setTimeZone(config.getTimeZone()); + + mDtHrFormatter = new SimpleDateFormat(mDtFormat+ "-" + mHrFormat); + mDtHrFormatter.setTimeZone(config.getTimeZone()); + + mDtHrMinFormatter = new SimpleDateFormat(mDtFormat+ "-" + mHrFormat + "-" + mMinFormat); + mDtHrMinFormatter.setTimeZone(config.getTimeZone()); + } + + static boolean usingHourly(SecorConfig config) { + return config.getBoolean("partitioner.granularity.hour", false); + } + + static boolean usingMinutely(SecorConfig config) { + return config.getBoolean("partitioner.granularity.minute", false); + } + + static String usingDateFormat(SecorConfig config) { + return config.getString("partitioner.granularity.date.format", "yyyy-MM-dd"); } - abstract protected long extractTimestampMillis(final Message message) throws Exception; + static String usingHourFormat(SecorConfig config) { + return config.getString("partitioner.granularity.hour.format", "HH"); + } + + static String usingMinuteFormat(SecorConfig config) { + return config.getString("partitioner.granularity.min.format", "mm"); + } + + static String usingDatePrefix(SecorConfig config) { + return config.getString("partitioner.granularity.date.prefix", "dt="); + } + + static String usingHourPrefix(SecorConfig config) { + return config.getString("partitioner.granularity.hour.prefix", "hr="); + } + + static String usingMinutePrefix(SecorConfig config) { + return config.getString("partitioner.granularity.minute.prefix", "min="); + } + + static boolean useKafkaTimestamp(SecorConfig config) { + return config.useKafkaTimestamp(); + } protected static long toMillis(final long timestamp) { final long nanosecondDivider = (long) Math.pow(10, 9 + 9); + final long microsecondDivider = (long) Math.pow(10, 9 + 6); 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 / microsecondDivider > 0L) { + timestampMillis = timestamp / (long) Math.pow(10, 3); } else if (timestamp / millisecondDivider > 0L) { timestampMillis = timestamp; } else { // assume seconds @@ -44,14 +147,134 @@ protected static long toMillis(final long timestamp) { return timestampMillis; } + public abstract long extractTimestampMillis(final Message message) throws Exception; + + public long getTimestampMillis(Message message) throws Exception { + return (mUseKafkaTimestamp) ? toMillis(message.getTimestamp()) : extractTimestampMillis(message); + } + + protected String[] generatePartitions(long timestampMillis, boolean usingHourly, boolean usingMinutely) + throws Exception { + Date date = new Date(timestampMillis); + String dt = mDtPrefix + mDtFormatter.format(date); + String hr = mHrPrefix + mHrFormatter.format(date); + String min = mMinPrefix + mMinFormatter.format(date); + if (usingMinutely) { + return new String[]{dt, hr, min}; + } else if (usingHourly) { + return new String[]{dt, hr}; + } else { + return new String[]{dt}; + } + } + + protected long parsePartitions(String[] partitions) throws Exception { + String dtValue = partitions[0].split("=")[1]; + String hrValue = partitions.length > 1 ? partitions[1].split("=")[1] : "00"; + String minValue = partitions.length > 2 ? partitions[2].split("=")[1] : "00"; + String value = dtValue + "-" + hrValue + "-" + minValue; + Date date = mDtHrMinFormatter.parse(value); + return date.getTime(); + } + @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; + long timestampMillis = getTimestampMillis(message); + return generatePartitions(timestampMillis, mUsingHourly, mUsingMinutely); + } + + private long getFinalizedTimestampMillis(Message lastMessage, + Message committedMessage) throws Exception { + long lastTimestamp = getTimestampMillis(lastMessage); + long committedTimestamp = getTimestampMillis(committedMessage); + long now = System.currentTimeMillis(); + if (lastTimestamp == committedTimestamp && + (now - lastTimestamp) > mFinalizerDelaySeconds * 1000) { + LOG.info("No new message coming, use the current time: " + now); + return now; + } + return committedTimestamp; + } + + @Override + public String[] getFinalizedUptoPartitions(List lastMessages, + List committedMessages) throws Exception { + + if (lastMessages == null || committedMessages == null) { + LOG.error("Either: {} and {} is null", lastMessages, committedMessages); + return null; + } + assert lastMessages.size() == committedMessages.size(); + + long minMillis = Long.MAX_VALUE; + for (int i = 0; i < lastMessages.size(); i++) { + long millis = getFinalizedTimestampMillis(lastMessages.get(i), committedMessages.get(i)); + if (millis < minMillis) { + LOG.info("partition {}, time {}", i, millis); + minMillis = millis; + } + } + if (minMillis == Long.MAX_VALUE) { + LOG.error("No valid timestamps among messages: {} and {}", lastMessages, committedMessages); + return null; + } + + // add the safety lag for late-arrival messages + minMillis -= mFinalizerDelaySeconds * 1000L; + LOG.info("adjusted millis {}", minMillis); + return generatePartitions(minMillis, mUsingHourly, mUsingMinutely); + } + + @Override + public String[] getPreviousPartitions(String[] partitions) throws Exception { + long millis = parsePartitions(partitions); + boolean usingHourly = mUsingHourly; + boolean usingMinutely = mUsingMinutely; + + if (mUsingMinutely && millis % HOUR_IN_MILLIS == 0) { + if (partitions.length == 3) { + usingMinutely = false; + if (millis % DAY_IN_MILLIS == 0) { + millis -= DAY_IN_MILLIS; + } else { + millis -= HOUR_IN_MILLIS; + usingHourly = true; + } + } else if (partitions.length == 2) { + millis += HOUR_IN_MILLIS; + millis -= MINUTE_IN_MILLIS; + usingMinutely = true; + } else { + millis += DAY_IN_MILLIS; + millis -= HOUR_IN_MILLIS; + usingMinutely = false; + usingHourly = true; + } + } else if (mUsingHourly && millis % DAY_IN_MILLIS == 0) { + // On the day boundary, if the current partition is [dt=07-07, hr=00], the previous + // one is dt=07-06; If the current one is [dt=07-06], the previous one is + // [dt=07-06, hr-23] + // So we would return in the order of: + // dt=07-07, hr=01 + // dt=07-07, hr=00 + // dt=07-06 + // dt=07-06, hr=23 + if (partitions.length == 2) { + usingHourly = false; + millis -= DAY_IN_MILLIS; + } else { + usingHourly = true; + millis += DAY_IN_MILLIS; + millis -= HOUR_IN_MILLIS; + } + } else { + long delta = mUsingHourly ? HOUR_IN_MILLIS : DAY_IN_MILLIS; + if (mUsingMinutely) { + delta = MINUTE_IN_MILLIS; + } + millis -= delta; + } + return generatePartitions(millis, usingHourly, usingMinutely); } } diff --git a/src/main/java/com/pinterest/secor/reader/MessageReader.java b/src/main/java/com/pinterest/secor/reader/MessageReader.java index f7964c3d8..b99813522 100644 --- a/src/main/java/com/pinterest/secor/reader/MessageReader.java +++ b/src/main/java/com/pinterest/secor/reader/MessageReader.java @@ -20,6 +20,7 @@ import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.common.TopicPartition; import com.pinterest.secor.message.Message; +import com.pinterest.secor.timestamp.KafkaMessageTimestampFactory; import com.pinterest.secor.util.IdUtil; import com.pinterest.secor.util.RateLimitUtil; import com.pinterest.secor.util.StatsUtil; @@ -29,6 +30,7 @@ import kafka.consumer.KafkaStream; import kafka.consumer.TopicFilter; import kafka.consumer.Whitelist; +import kafka.consumer.Blacklist; import kafka.javaapi.consumer.ConsumerConnector; import kafka.message.MessageAndMetadata; import org.slf4j.Logger; @@ -49,11 +51,15 @@ 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; + protected SecorConfig mConfig; + protected OffsetTracker mOffsetTracker; + protected ConsumerConnector mConsumerConnector; + protected ConsumerIterator mIterator; + protected HashMap mLastAccessTime; + protected final int mTopicPartitionForgetSeconds; + protected final int mCheckMessagesPerSecond; + protected int mNMessages; + protected KafkaMessageTimestampFactory mKafkaMessageTimestampFactory; public MessageReader(SecorConfig config, OffsetTracker offsetTracker) throws UnknownHostException { @@ -62,13 +68,21 @@ public MessageReader(SecorConfig config, OffsetTracker offsetTracker) throws mConsumerConnector = Consumer.createJavaConsumerConnector(createConsumerConfig()); - TopicFilter topicFilter = new Whitelist(mConfig.getKafkaTopicFilter()); + if (!mConfig.getKafkaTopicBlacklist().isEmpty() && !mConfig.getKafkaTopicFilter().isEmpty()) { + throw new RuntimeException("Topic filter and blacklist cannot be both specified."); + } + TopicFilter topicFilter = !mConfig.getKafkaTopicBlacklist().isEmpty()? new Blacklist(mConfig.getKafkaTopicBlacklist()): + new Whitelist(mConfig.getKafkaTopicFilter()); + LOG.debug("Use TopicFilter {}({})", topicFilter.getClass(), topicFilter); List> streams = mConsumerConnector.createMessageStreamsByFilter(topicFilter); KafkaStream stream = streams.get(0); mIterator = stream.iterator(); mLastAccessTime = new HashMap(); StatsUtil.setLabel("secor.kafka.consumer.id", IdUtil.getConsumerId()); + mTopicPartitionForgetSeconds = mConfig.getTopicPartitionForgetSeconds(); + mCheckMessagesPerSecond = mConfig.getMessagesPerSecond() / mConfig.getConsumerThreads(); + mKafkaMessageTimestampFactory = new KafkaMessageTimestampFactory(mConfig.getKafkaMessageTimestampClass()); } private void updateAccessTime(TopicPartition topicPartition) { @@ -78,7 +92,7 @@ private void updateAccessTime(TopicPartition topicPartition) { while (iterator.hasNext()) { Map.Entry pair = (Map.Entry) iterator.next(); long lastAccessTime = (Long) pair.getValue(); - if (now - lastAccessTime > mConfig.getTopicPartitionForgetSeconds()) { + if (now - lastAccessTime > mTopicPartitionForgetSeconds) { iterator.remove(); } } @@ -105,20 +119,35 @@ private ConsumerConfig createConsumerConfig() throws UnknownHostException { 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("auto.offset.reset", mConfig.getConsumerAutoOffsetReset()); props.put("consumer.timeout.ms", Integer.toString(mConfig.getConsumerTimeoutMs())); props.put("consumer.id", IdUtil.getConsumerId()); - if (mConfig.getRebalanceMaxRetries() != null && !mConfig.getRebalanceMaxRetries().isEmpty()) { + // Properties required to upgrade from kafka 0.8.x to 0.9.x + props.put("dual.commit.enabled", mConfig.getDualCommitEnabled()); + props.put("offsets.storage", mConfig.getOffsetsStorage()); + + props.put("partition.assignment.strategy", mConfig.getPartitionAssignmentStrategy()); + if (mConfig.getRebalanceMaxRetries() != null && + !mConfig.getRebalanceMaxRetries().isEmpty()) { props.put("rebalance.max.retries", mConfig.getRebalanceMaxRetries()); } - if (mConfig.getSocketReceieveBufferBytes() != null && !mConfig.getSocketReceieveBufferBytes().isEmpty()) { - props.put("socket.receive.buffer.bytes", mConfig.getSocketReceieveBufferBytes()); + if (mConfig.getRebalanceBackoffMs() != null && + !mConfig.getRebalanceBackoffMs().isEmpty()) { + props.put("rebalance.backoff.ms", mConfig.getRebalanceBackoffMs()); + } + if (mConfig.getSocketReceiveBufferBytes() != null && + !mConfig.getSocketReceiveBufferBytes().isEmpty()) { + props.put("socket.receive.buffer.bytes", mConfig.getSocketReceiveBufferBytes()); } if (mConfig.getFetchMessageMaxBytes() != null && !mConfig.getFetchMessageMaxBytes().isEmpty()) { props.put("fetch.message.max.bytes", mConfig.getFetchMessageMaxBytes()); } + if (mConfig.getFetchMinBytes() != null && !mConfig.getFetchMinBytes().isEmpty()) { + props.put("fetch.min.bytes", mConfig.getFetchMinBytes()); + } + if (mConfig.getFetchWaitMaxMs() != null && !mConfig.getFetchWaitMaxMs().isEmpty()) { + props.put("fetch.wait.max.ms", mConfig.getFetchWaitMaxMs()); + } return new ConsumerConfig(props); } @@ -129,20 +158,30 @@ public boolean hasNext() { public Message read() { assert hasNext(); - RateLimitUtil.acquire(); + mNMessages = (mNMessages + 1) % mCheckMessagesPerSecond; + if (mNMessages % mCheckMessagesPerSecond == 0) { + RateLimitUtil.acquire(mCheckMessagesPerSecond); + } MessageAndMetadata kafkaMessage = mIterator.next(); + + long timestamp = (mConfig.useKafkaTimestamp()) + ? mKafkaMessageTimestampFactory.getKafkaMessageTimestamp().getTimestamp(kafkaMessage) + : 0l; Message message = new Message(kafkaMessage.topic(), kafkaMessage.partition(), - kafkaMessage.offset(), kafkaMessage.message()); + kafkaMessage.offset(), kafkaMessage.key(), + kafkaMessage.message(), timestamp); 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(); + LOG.debug("read message {}", message); + if (mNMessages % mCheckMessagesPerSecond == 0) { + exportStats(); + } if (message.getOffset() < committedOffsetCount) { - LOG.debug("skipping message message " + message + " because its offset precedes " + - "committed offset count " + committedOffsetCount); + LOG.debug("skipping message {} because its offset precedes committed offset count {}", + message, committedOffsetCount); return null; } return message; diff --git a/src/main/java/com/pinterest/secor/timestamp/Kafka10MessageTimestamp.java b/src/main/java/com/pinterest/secor/timestamp/Kafka10MessageTimestamp.java new file mode 100644 index 000000000..043901ef5 --- /dev/null +++ b/src/main/java/com/pinterest/secor/timestamp/Kafka10MessageTimestamp.java @@ -0,0 +1,17 @@ +package com.pinterest.secor.timestamp; + +import kafka.message.MessageAndMetadata; +import kafka.message.MessageAndOffset; + +public class Kafka10MessageTimestamp implements KafkaMessageTimestamp { + + @Override + public long getTimestamp(MessageAndMetadata kafkaMessage) { + return kafkaMessage.timestamp(); + } + + @Override + public long getTimestamp(MessageAndOffset messageAndOffset) { + return messageAndOffset.message().timestamp(); + } +} diff --git a/src/main/java/com/pinterest/secor/timestamp/Kafka8MessageTimestamp.java b/src/main/java/com/pinterest/secor/timestamp/Kafka8MessageTimestamp.java new file mode 100644 index 000000000..bc461734e --- /dev/null +++ b/src/main/java/com/pinterest/secor/timestamp/Kafka8MessageTimestamp.java @@ -0,0 +1,17 @@ +package com.pinterest.secor.timestamp; + +import kafka.message.MessageAndMetadata; +import kafka.message.MessageAndOffset; + +public class Kafka8MessageTimestamp implements KafkaMessageTimestamp { + + @Override + public long getTimestamp(MessageAndMetadata kafkaMessage) { + return 0l; + } + + @Override + public long getTimestamp(MessageAndOffset messageAndOffset) { + return 0l; + } +} diff --git a/src/main/java/com/pinterest/secor/timestamp/KafkaMessageTimestamp.java b/src/main/java/com/pinterest/secor/timestamp/KafkaMessageTimestamp.java new file mode 100644 index 000000000..4d53b13a9 --- /dev/null +++ b/src/main/java/com/pinterest/secor/timestamp/KafkaMessageTimestamp.java @@ -0,0 +1,11 @@ +package com.pinterest.secor.timestamp; + +import kafka.message.MessageAndMetadata; +import kafka.message.MessageAndOffset; + +public interface KafkaMessageTimestamp { + + long getTimestamp(MessageAndMetadata kafkaMessage); + + long getTimestamp(MessageAndOffset messageAndOffset); +} diff --git a/src/main/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactory.java b/src/main/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactory.java new file mode 100644 index 000000000..cc9a760cc --- /dev/null +++ b/src/main/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactory.java @@ -0,0 +1,23 @@ +package com.pinterest.secor.timestamp; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class KafkaMessageTimestampFactory { + private static final Logger LOG = LoggerFactory.getLogger(KafkaMessageTimestampFactory.class); + + private KafkaMessageTimestamp kafkaMessageTimestamp; + + public KafkaMessageTimestampFactory(String kafkaTimestampClassName) { + try { + Class timestampClass = Class.forName(kafkaTimestampClassName); + this.kafkaMessageTimestamp = KafkaMessageTimestamp.class.cast(timestampClass.newInstance()); + } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) { + throw new RuntimeException(e); + } + } + + public KafkaMessageTimestamp getKafkaMessageTimestamp() { + return this.kafkaMessageTimestamp; + } +} diff --git a/src/main/java/com/pinterest/secor/tools/LogFileDeleter.java b/src/main/java/com/pinterest/secor/tools/LogFileDeleter.java index 68535da85..e12ada564 100644 --- a/src/main/java/com/pinterest/secor/tools/LogFileDeleter.java +++ b/src/main/java/com/pinterest/secor/tools/LogFileDeleter.java @@ -23,7 +23,6 @@ import java.io.IOException; import java.text.SimpleDateFormat; -import java.util.TimeZone; /** * Log file deleter removes message old log files stored locally. @@ -44,17 +43,15 @@ public void deleteOldLogs() throws Exception { } String[] consumerDirs = FileUtil.list(mConfig.getLocalPath()); SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss z"); - format.setTimeZone(TimeZone.getTimeZone("UTC")); + format.setTimeZone(mConfig.getTimeZone()); for (String consumerDir : consumerDirs) { long modificationTime = FileUtil.getModificationTimeMsRecursive(consumerDir); String modificationTimeStr = format.format(modificationTime); - LOG.info("Consumer log dir " + consumerDir + " last modified at " + - modificationTimeStr); + LOG.info("Consumer log dir {} last modified at {}", consumerDir , modificationTimeStr); final long localLogDeleteAgeMs = mConfig.getLocalLogDeleteAgeHours() * 60L * 60L * 1000L; if (System.currentTimeMillis() - modificationTime > localLogDeleteAgeMs) { - LOG.info("Deleting directory " + consumerDir + " last modified at " + - modificationTimeStr); + LOG.info("Deleting directory {} last modified at {}", consumerDir, modificationTimeStr); FileUtil.delete(consumerDir); } } diff --git a/src/main/java/com/pinterest/secor/tools/LogFilePrinter.java b/src/main/java/com/pinterest/secor/tools/LogFilePrinter.java index 22083c1c7..9e947c2b3 100644 --- a/src/main/java/com/pinterest/secor/tools/LogFilePrinter.java +++ b/src/main/java/com/pinterest/secor/tools/LogFilePrinter.java @@ -50,7 +50,9 @@ public void printFile(String path) throws Exception { if (mPrintOffsetsOnly) { System.out.println(Long.toString(key.get())); } else { - System.out.println(Long.toString(key.get()) + ": " + new String(value.getBytes())); + byte[] nonPaddedBytes = new byte[value.getLength()]; + System.arraycopy(value.getBytes(), 0, nonPaddedBytes, 0, value.getLength()); + System.out.println(Long.toString(key.get()) + ": " + new String(nonPaddedBytes)); } } } diff --git a/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java b/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java index 681374d1b..949d0c0c8 100644 --- a/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java +++ b/src/main/java/com/pinterest/secor/tools/LogFileVerifier.java @@ -19,13 +19,13 @@ import com.pinterest.secor.common.LogFilePath; import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.common.TopicPartition; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.KeyValue; +import com.pinterest.secor.util.CompressionUtil; 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 com.pinterest.secor.util.ReflectionUtil; + +import org.apache.hadoop.io.compress.CompressionCodec; import java.io.IOException; import java.util.*; @@ -48,16 +48,12 @@ public LogFileVerifier(SecorConfig config, String topic) throws IOException { new HashMap>>(); } - private String getPrefix() { - return "s3n://" + mConfig.getS3Bucket() + "/" + mConfig.getS3Path(); - } - - private String getTopicPrefix() { - return getPrefix() + "/" + mTopic; + private String getTopicPrefix() throws IOException { + return FileUtil.getPrefix(mTopic, mConfig) + "/" + mTopic; } private void populateTopicPartitionToOffsetToFiles() throws IOException { - String prefix = getPrefix(); + String prefix = FileUtil.getPrefix(mTopic, mConfig); String topicPrefix = getTopicPrefix(); String[] paths = FileUtil.listRecursively(topicPrefix); for (String path : paths) { @@ -107,15 +103,9 @@ private void filterOffsets(long fromOffset, long toOffset) { } 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(); + FileReader reader = createFileReader(logFilePath); int result = 0; - while (reader.next(key, value)) { + while (reader.next() != null) { result++; } reader.close(); @@ -162,16 +152,11 @@ public void verifyCounts(long fromOffset, long toOffset, int numMessages) throws } 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 " + + FileReader reader = createFileReader(logFilePath); + KeyValue record; + while ((record = reader.next()) != null) { + if (!offsets.add(record.getOffset())) { + throw new RuntimeException("duplicate key " + record.getOffset() + " found in file " + logFilePath.getLogFilePath()); } } @@ -205,4 +190,25 @@ public void verifySequences(long fromOffset, long toOffset) throws Exception { } } } + + /** + * Helper to create a file reader writer from config + * + * @param logFilePath + * @return + * @throws Exception + */ + private FileReader createFileReader(LogFilePath logFilePath) throws Exception { + CompressionCodec codec = null; + if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { + codec = CompressionUtil.createCompressionCodec(mConfig.getCompressionCodec()); + } + FileReader fileReader = ReflectionUtil.createFileReader( + mConfig.getFileReaderWriterFactory(), + logFilePath, + codec, + mConfig + ); + return fileReader; + } } diff --git a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java index 3491db350..4d44d3bf3 100644 --- a/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java +++ b/src/main/java/com/pinterest/secor/tools/ProgressMonitor.java @@ -16,21 +16,35 @@ */ package com.pinterest.secor.tools; +import com.google.common.base.Joiner; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.net.HostAndPort; 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.parser.MessageParser; +import com.pinterest.secor.parser.TimestampedMessageParser; +import com.pinterest.secor.util.ReflectionUtil; +import com.timgroup.statsd.NonBlockingStatsDClient; + +import net.minidev.json.JSONArray; import net.minidev.json.JSONObject; import net.minidev.json.JSONValue; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.*; +import java.io.BufferedReader; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; import java.net.HttpURLConnection; import java.net.URL; -import java.util.HashMap; import java.util.List; import java.util.Map; @@ -40,17 +54,35 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ public class ProgressMonitor { + private static final Logger LOG = LoggerFactory.getLogger(ProgressMonitor.class); + private static final String PERIOD = "."; + private SecorConfig mConfig; private ZookeeperConnector mZookeeperConnector; private KafkaClient mKafkaClient; - private ThriftMessageParser mThriftMessageParser; + private MessageParser mMessageParser; + private String mPrefix; + private NonBlockingStatsDClient mStatsDClient; - public ProgressMonitor(SecorConfig config) { + public ProgressMonitor(SecorConfig config) + throws Exception + { mConfig = config; mZookeeperConnector = new ZookeeperConnector(mConfig); mKafkaClient = new KafkaClient(mConfig); - mThriftMessageParser = new ThriftMessageParser(mConfig); + mMessageParser = (MessageParser) ReflectionUtil.createMessageParser( + mConfig.getMessageParserClass(), mConfig); + + mPrefix = mConfig.getMonitoringPrefix(); + if (Strings.isNullOrEmpty(mPrefix)) { + mPrefix = "secor"; + } + + if (mConfig.getStatsDHostPort() != null && !mConfig.getStatsDHostPort().isEmpty()) { + HostAndPort hostPort = HostAndPort.fromString(mConfig.getStatsDHostPort()); + mStatsDClient = new NonBlockingStatsDClient(null, hostPort.getHostText(), hostPort.getPort()); + } } private void makeRequest(String body) throws IOException { @@ -95,27 +127,62 @@ private void makeRequest(String body) throws IOException { } } - private void exportToTsdb(String metric, Map tags, String value) + private void exportToTsdb(Stat stat) 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()); + LOG.info("exporting metric to tsdb {}", stat); + makeRequest(stat.toString()); } public void exportStats() throws Exception { + List stats = getStats(); + LOG.info("Stats: {}", JSONArray.toJSONString(stats)); + + // if there is a valid openTSDB port configured export to openTSDB + if (mConfig.getTsdbHostport() != null && !mConfig.getTsdbHostport().isEmpty()) { + for (Stat stat : stats) { + exportToTsdb(stat); + } + } + + // if there is a valid statsD port configured export to statsD + if (mStatsDClient != null) { + exportToStatsD(stats); + } + } + + /** + * Helper to publish stats to statsD client + */ + private void exportToStatsD(List stats) { + // group stats by kafka group + for (Stat stat : stats) { + @SuppressWarnings("unchecked") + Map tags = (Map) stat.get(Stat.STAT_KEYS.TAGS.getName()); + StringBuilder builder = new StringBuilder(); + if (mConfig.getStatsDPrefixWithConsumerGroup()) { + builder.append(tags.get(Stat.STAT_KEYS.GROUP.getName())) + .append(PERIOD); + } + String aspect = builder + .append((String)stat.get(Stat.STAT_KEYS.METRIC.getName())) + .append(PERIOD) + .append(tags.get(Stat.STAT_KEYS.TOPIC.getName())) + .append(PERIOD) + .append(tags.get(Stat.STAT_KEYS.PARTITION.getName())) + .toString(); + long value = Long.parseLong((String)stat.get(Stat.STAT_KEYS.VALUE.getName())); + mStatsDClient.recordGaugeValue(aspect, value); + } + } + + private List getStats() throws Exception { List topics = mZookeeperConnector.getCommittedOffsetTopics(); + List stats = Lists.newArrayList(); + for (String topic : topics) { - if (topic.matches(mConfig.getTsdbBlacklistTopics()) || + if (topic.matches(mConfig.getMonitoringBlacklistTopics()) || !topic.matches(mConfig.getKafkaTopicFilter())) { - LOG.info("skipping topic " + topic); + LOG.info("skipping topic {}", topic); continue; } List partitions = mZookeeperConnector.getCommittedOffsetPartitions(topic); @@ -125,37 +192,96 @@ public void exportStats() throws Exception { long committedOffset = - 1; long committedTimestampMillis = -1; if (committedMessage == null) { - LOG.warn("no committed message found in topic " + topic + " partition " + - partition); + LOG.warn("no committed message found in topic {} partition {}", topic, partition); + continue; } else { committedOffset = committedMessage.getOffset(); - committedTimestampMillis = mThriftMessageParser.extractTimestampMillis( - committedMessage); + committedTimestampMillis = getTimestamp(committedMessage); } Message lastMessage = mKafkaClient.getLastMessage(topicPartition); if (lastMessage == null) { - LOG.warn("no message found in topic " + topic + " partition " + partition); + LOG.warn("no message found in topic {} partition {}", topic, partition); } else { long lastOffset = lastMessage.getOffset(); - long lastTimestampMillis = mThriftMessageParser.extractTimestampMillis( - lastMessage); + long lastTimestampMillis = getTimestamp(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)); + Map tags = ImmutableMap.of( + Stat.STAT_KEYS.TOPIC.getName(), topic, + Stat.STAT_KEYS.PARTITION.getName(), Integer.toString(partition), + Stat.STAT_KEYS.GROUP.getName(), mConfig.getKafkaGroup() + ); + + long timestamp = System.currentTimeMillis() / 1000; + stats.add(Stat.createInstance(metricName("lag.offsets"), tags, Long.toString(offsetLag), timestamp)); + stats.add(Stat.createInstance(metricName("lag.seconds"), tags, Long.toString(timestampMillisLag / 1000), timestamp)); + + LOG.debug("topic {} partition {} committed offset {} last offset {} committed timestamp {} last timestamp {}", + topic, partition, committedOffset, lastOffset, + (committedTimestampMillis / 1000), (lastTimestampMillis / 1000)); } } } + + return stats; + } + + private String metricName(String key) { + return Joiner.on(".").join(mPrefix, key); } + + private long getTimestamp(Message message) throws Exception { + if (mMessageParser instanceof TimestampedMessageParser) { + return ((TimestampedMessageParser)mMessageParser).getTimestampMillis(message); + } else { + return -1; + } + } + + /** + * + * JSON hash map extension to store statistics + * + */ + private static class Stat extends JSONObject { + + // definition of all the stat keys + public enum STAT_KEYS { + METRIC("metric"), + TAGS("tags"), + VALUE("value"), + TIMESTAMP("timestamp"), + TOPIC("topic"), + PARTITION("partition"), + GROUP("group"); + + STAT_KEYS(String name) { + this.mName = name; + } + + private final String mName; + + public String getName() { + return this.mName; + } + } + + public static Stat createInstance(String metric, Map tags, String value, long timestamp) { + return new Stat(ImmutableMap.of( + STAT_KEYS.METRIC.getName(), metric, + STAT_KEYS.TAGS.getName(), tags, + STAT_KEYS.VALUE.getName(), value, + STAT_KEYS.TIMESTAMP.getName(), timestamp + )); + } + + public Stat(Map map) { + super(map); + } + } + } diff --git a/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java b/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java index e85143b3b..a8eb86023 100644 --- a/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java +++ b/src/main/java/com/pinterest/secor/tools/TestLogMessageProducer.java @@ -19,9 +19,13 @@ 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 org.apache.thrift.protocol.TProtocolFactory; +import org.apache.thrift.protocol.TSimpleJSONProtocol; + import com.pinterest.secor.thrift.TestMessage; import com.pinterest.secor.thrift.TestEnum; @@ -33,17 +37,28 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ public class TestLogMessageProducer extends Thread { - private String mTopic; - private int mNumMessages; + private final String mTopic; + private final int mNumMessages; + private final String mType; + private final String mMetadataBrokerList; + private final int mTimeshift; - public TestLogMessageProducer(String topic, int numMessages) { + public TestLogMessageProducer(String topic, int numMessages, String type, + String metadataBrokerList, int timeshift) { mTopic = topic; mNumMessages = numMessages; + mType = type; + mMetadataBrokerList = metadataBrokerList; + mTimeshift = timeshift; } public void run() { Properties properties = new Properties(); - properties.put("metadata.broker.list", "localhost:9092"); + if (mMetadataBrokerList == null || mMetadataBrokerList.isEmpty()) { + properties.put("metadata.broker.list", "localhost:9092"); + } else { + properties.put("metadata.broker.list", mMetadataBrokerList); + } properties.put("partitioner.class", "com.pinterest.secor.tools.RandomPartitioner"); properties.put("serializer.class", "kafka.serializer.DefaultEncoder"); properties.put("key.serializer.class", "kafka.serializer.StringEncoder"); @@ -52,9 +67,19 @@ public void run() { ProducerConfig config = new ProducerConfig(properties); Producer producer = new Producer(config); - TSerializer serializer = new TSerializer(new TBinaryProtocol.Factory()); + TProtocolFactory protocol = null; + if(mType.equals("json")) { + protocol = new TSimpleJSONProtocol.Factory(); + } else if (mType.equals("binary")) { + protocol = new TBinaryProtocol.Factory(); + } else { + throw new RuntimeException("Undefined message encoding type: " + mType); + } + + TSerializer serializer = new TSerializer(protocol); for (int i = 0; i < mNumMessages; ++i) { - TestMessage testMessage = new TestMessage(System.currentTimeMillis() * 1000000L + i, + long time = (System.currentTimeMillis() - mTimeshift * 1000L) * 1000000L + i; + TestMessage testMessage = new TestMessage(time, "some_value_" + i); if (i % 2 == 0) { testMessage.setEnumField(TestEnum.SOME_VALUE); @@ -73,4 +98,4 @@ public void run() { } producer.close(); } -} +} \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/transformer/IdentityMessageTransformer.java b/src/main/java/com/pinterest/secor/transformer/IdentityMessageTransformer.java new file mode 100644 index 000000000..236d239f2 --- /dev/null +++ b/src/main/java/com/pinterest/secor/transformer/IdentityMessageTransformer.java @@ -0,0 +1,26 @@ +package com.pinterest.secor.transformer; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; + +/** + * Default message transformer class which does no transformation + * + * @author Ashish (ashu.impetus@gmail.com) + * + */ +public class IdentityMessageTransformer implements MessageTransformer { + protected SecorConfig mConfig; + /** + * Constructor + * + * @param config + */ + public IdentityMessageTransformer(SecorConfig config) { + mConfig = config; + } + @Override + public Message transform(Message message) { + return message; + } +} diff --git a/src/main/java/com/pinterest/secor/transformer/MessageTransformer.java b/src/main/java/com/pinterest/secor/transformer/MessageTransformer.java new file mode 100644 index 000000000..d03d89671 --- /dev/null +++ b/src/main/java/com/pinterest/secor/transformer/MessageTransformer.java @@ -0,0 +1,17 @@ +package com.pinterest.secor.transformer; + +import com.pinterest.secor.message.Message; + +/** + * Message transformer Interface + * + * @author Ashish (ashu.impetus@gmail.com) + * + */ +public interface MessageTransformer { + /** + * Implement this method to add transformation logic at message level before + * dumping it into Amazon S3 + */ + public Message transform(Message message); +} diff --git a/src/main/java/com/pinterest/secor/uploader/AzureUploadManager.java b/src/main/java/com/pinterest/secor/uploader/AzureUploadManager.java new file mode 100644 index 000000000..412ab6341 --- /dev/null +++ b/src/main/java/com/pinterest/secor/uploader/AzureUploadManager.java @@ -0,0 +1,78 @@ +package com.pinterest.secor.uploader; + + +import com.microsoft.azure.storage.CloudStorageAccount; +import com.microsoft.azure.storage.StorageException; +import com.microsoft.azure.storage.blob.CloudBlobClient; +import com.microsoft.azure.storage.blob.CloudBlobContainer; +import com.microsoft.azure.storage.blob.CloudBlockBlob; +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.common.SecorConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URISyntaxException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +/** + * Manages uploads to Microsoft Azure blob storage using Azure Storage SDK for java + * https://github.com/azure/azure-storage-java + * + * @author Taichi Nakashima (nsd22843@gmail.com) + * + */ +public class AzureUploadManager extends UploadManager { + private static final Logger LOG = LoggerFactory.getLogger(AzureUploadManager.class); + private static final ExecutorService executor = Executors.newFixedThreadPool(256); + + private CloudBlobClient blobClient; + + public AzureUploadManager(SecorConfig config) throws Exception { + super(config); + + final String storageConnectionString = + "DefaultEndpointsProtocol=" + mConfig.getAzureEndpointsProtocol() + ";" + + "AccountName=" + mConfig.getAzureAccountName() + ";" + + "AccountKey=" + mConfig.getAzureAccountKey() + ";"; + + CloudStorageAccount storageAccount = CloudStorageAccount.parse(storageConnectionString); + blobClient = storageAccount.createCloudBlobClient(); + } + + @java.lang.Override + public Handle upload(LogFilePath localPath) throws Exception { + final String azureContainer = mConfig.getAzureContainer(); + final String azureKey = localPath.withPrefix(mConfig.getAzurePath()).getLogFilePath(); + final File localFile = new File(localPath.getLogFilePath()); + + LOG.info("uploading file {} to azure://{}/{}", localFile, azureContainer, azureKey); + final Future f = executor.submit(new Runnable() { + @Override + public void run() { + try { + CloudBlobContainer container = blobClient.getContainerReference(azureContainer); + container.createIfNotExists(); + + CloudBlockBlob blob = container.getBlockBlobReference(azureKey); + blob.upload(new java.io.FileInputStream(localFile), localFile.length()); + + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } catch (StorageException e) { + throw new RuntimeException(e); + } catch (FileNotFoundException e) { + throw new RuntimeException(e); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }); + + return new FutureHandle(f); + } +} diff --git a/src/main/java/com/pinterest/secor/uploader/FutureHandle.java b/src/main/java/com/pinterest/secor/uploader/FutureHandle.java new file mode 100644 index 000000000..ad3be712a --- /dev/null +++ b/src/main/java/com/pinterest/secor/uploader/FutureHandle.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.pinterest.secor.uploader; + +import java.util.concurrent.Future; + +/** + * Wraps a Future. `get` blocks until the underlying Future completes. + * + * @author Liam Stewart (liam.stewart@gmail.com) + */ +public class FutureHandle implements Handle { + private Future mFuture; + + public FutureHandle(Future f) { + mFuture = f; + } + + public T get() throws Exception { + return mFuture.get(); + } +} diff --git a/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java new file mode 100644 index 000000000..b7e3a175d --- /dev/null +++ b/src/main/java/com/pinterest/secor/uploader/GsUploadManager.java @@ -0,0 +1,156 @@ +package com.pinterest.secor.uploader; + +import com.google.api.client.googleapis.auth.oauth2.GoogleCredential; +import com.google.api.client.googleapis.javanet.GoogleNetHttpTransport; +import com.google.api.client.googleapis.media.MediaHttpUploader; +import com.google.api.client.googleapis.media.MediaHttpUploaderProgressListener; +import com.google.api.client.http.FileContent; +import com.google.api.client.http.HttpRequest; +import com.google.api.client.http.HttpRequestInitializer; +import com.google.api.client.http.HttpTransport; +import com.google.api.client.http.HttpUnsuccessfulResponseHandler; +import com.google.api.client.http.HttpBackOffUnsuccessfulResponseHandler; +import com.google.api.client.json.JsonFactory; +import com.google.api.client.json.jackson2.JacksonFactory; +import com.google.api.client.util.ExponentialBackOff; +import com.google.api.services.storage.Storage; +import com.google.api.services.storage.StorageScopes; +import com.google.api.services.storage.model.StorageObject; +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.common.SecorConfig; +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.nio.file.Files; +import java.util.Collections; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.Date; +import java.text.SimpleDateFormat; + +/** + * Manages uploads to Google Cloud Storage using the Storage class from the Google API SDK. + *

+ * It will use Service Account credential (json file) that can be generated from the Google Developers Console. + * By default it will look up configured credential path in secor.gs.credentials.path or fallback to the default + * credential in the environment variable GOOGLE_APPLICATION_CREDENTIALS. + *

+ * Application credentials documentation + * https://developers.google.com/identity/protocols/application-default-credentials + * + * @author Jerome Gagnon (jerome.gagnon.1@gmail.com) + */ +public class GsUploadManager extends UploadManager { + private static final Logger LOG = LoggerFactory.getLogger(GsUploadManager.class); + + private static final JsonFactory JSON_FACTORY = JacksonFactory.getDefaultInstance(); + + private static final ExecutorService executor = Executors.newFixedThreadPool(256); + + /** + * Global instance of the Storage. The best practice is to make it a single + * globally shared instance across your application. + */ + private static Storage mStorageService; + + private Storage mClient; + + public GsUploadManager(SecorConfig config) throws Exception { + super(config); + + mClient = getService(mConfig.getGsCredentialsPath(), + mConfig.getGsConnectTimeoutInMs(), mConfig.getGsReadTimeoutInMs()); + } + + @Override + public Handle upload(LogFilePath localPath) throws Exception { + final String gsBucket = mConfig.getGsBucket(); + final String gsKey = localPath.withPrefix(mConfig.getGsPath()).getLogFilePath(); + final File localFile = new File(localPath.getLogFilePath()); + final boolean directUpload = mConfig.getGsDirectUpload(); + + LOG.info("uploading file {} to gs://{}/{}", localFile, gsBucket, gsKey); + + final StorageObject storageObject = new StorageObject().setName(gsKey); + final FileContent storageContent = new FileContent(Files.probeContentType(localFile.toPath()), localFile); + + final Future f = executor.submit(new Runnable() { + @Override + public void run() { + try { + Storage.Objects.Insert request = mClient.objects().insert(gsBucket, storageObject, storageContent); + + if (directUpload) { + request.getMediaHttpUploader().setDirectUploadEnabled(true); + } + + request.getMediaHttpUploader().setProgressListener(new MediaHttpUploaderProgressListener() { + @Override + public void progressChanged(MediaHttpUploader uploader) throws IOException { + LOG.debug("[{} %] upload file {} to gs://{}/{}", + (int) uploader.getProgress() * 100, localFile, gsBucket, gsKey); + } + }); + + request.execute(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }); + + return new FutureHandle(f); + } + + private static Storage getService(String credentialsPath, int connectTimeoutMs, int readTimeoutMs) throws Exception { + if (mStorageService == null) { + HttpTransport httpTransport = GoogleNetHttpTransport.newTrustedTransport(); + + GoogleCredential credential; + try { + // Lookup if configured path from the properties; otherwise fallback to Google Application default + if (credentialsPath != null && !credentialsPath.isEmpty()) { + credential = GoogleCredential + .fromStream(new FileInputStream(credentialsPath), httpTransport, JSON_FACTORY) + .createScoped(Collections.singleton(StorageScopes.CLOUD_PLATFORM)); + } else { + credential = GoogleCredential.getApplicationDefault(httpTransport, JSON_FACTORY); + } + } catch (IOException e) { + throw new RuntimeException("Failed to load Google credentials : " + credentialsPath, e); + } + + mStorageService = new Storage.Builder(httpTransport, JSON_FACTORY, + setHttpBackoffTimeout(credential, connectTimeoutMs, readTimeoutMs)) + .setApplicationName("com.pinterest.secor") + .build(); + } + return mStorageService; + } + + private static HttpRequestInitializer setHttpBackoffTimeout(final HttpRequestInitializer requestInitializer, + final int connectTimeoutMs, final int readTimeoutMs) { + return new HttpRequestInitializer() { + @Override + public void initialize(HttpRequest httpRequest) throws IOException { + requestInitializer.initialize(httpRequest); + + // Configure exponential backoff on error + // https://developers.google.com/api-client-library/java/google-http-java-client/backoff + ExponentialBackOff backoff = new ExponentialBackOff(); + HttpUnsuccessfulResponseHandler backoffHandler = new HttpBackOffUnsuccessfulResponseHandler(backoff) + .setBackOffRequired(HttpBackOffUnsuccessfulResponseHandler.BackOffRequired.ALWAYS); + httpRequest.setUnsuccessfulResponseHandler(backoffHandler); + + httpRequest.setConnectTimeout(connectTimeoutMs); + httpRequest.setReadTimeout(readTimeoutMs); + } + }; + } + +} diff --git a/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java new file mode 100644 index 000000000..32662fd92 --- /dev/null +++ b/src/main/java/com/pinterest/secor/uploader/HadoopS3UploadManager.java @@ -0,0 +1,77 @@ +/** + * 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 java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.*; +import java.util.concurrent.Future; + +/** + * Manages uploads to S3 using the Hadoop API. + * + * @author Pawel Garbacki (pawel@pinterest.com) + */ +public class HadoopS3UploadManager extends UploadManager { + private static final Logger LOG = LoggerFactory.getLogger(HadoopS3UploadManager.class); + + protected static final ExecutorService executor = Executors.newFixedThreadPool(256); + + public HadoopS3UploadManager(SecorConfig config) { + super(config); + } + + public Handle upload(LogFilePath localPath) throws Exception { + String prefix = FileUtil.getPrefix(localPath.getTopic(), mConfig); + LogFilePath path = localPath.withPrefix(prefix); + path.setCustomTopicsNames(mConfig.getCustomTopicsNames()); + final String localLogFilename = localPath.getLogFilePath(); + final String logFileName; + + if (FileUtil.s3PathPrefixIsAltered(path.getLogFilePath(), mConfig)) { + logFileName = localPath.withPrefix(FileUtil.getS3AlternativePrefix(mConfig)) + .getLogFilePath(); + LOG.info("Will upload file to alternative s3 prefix path {}", logFileName); + } + else { + logFileName = path.getLogFilePath(); + } + + LOG.info("uploading file {} to {}", localLogFilename, logFileName); + + final Future f = executor.submit(new Runnable() { + @Override + public void run() { + try { + FileUtil.moveToCloud(localLogFilename, logFileName); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }); + + return new FutureHandle(f); + } +} diff --git a/src/main/java/com/pinterest/secor/uploader/Handle.java b/src/main/java/com/pinterest/secor/uploader/Handle.java new file mode 100644 index 000000000..c8da4a13b --- /dev/null +++ b/src/main/java/com/pinterest/secor/uploader/Handle.java @@ -0,0 +1,26 @@ +/** + * 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; + +/** + * Simple generic wrapper interface. + * + * @author Liam Stewart (liam.stewart@gmail.com) + */ +public interface Handle { + public T get() throws Exception; +} diff --git a/src/main/java/com/pinterest/secor/uploader/S3UploadHandle.java b/src/main/java/com/pinterest/secor/uploader/S3UploadHandle.java new file mode 100644 index 000000000..2e4aed691 --- /dev/null +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadHandle.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.uploader; + +import com.amazonaws.services.s3.transfer.Upload; +import com.amazonaws.services.s3.transfer.model.UploadResult; + +/** + * Wraps an Upload being managed by the AWS SDK TransferManager. `get` + * blocks until the upload completes. + * + * @author Liam Stewart (liam.stewart@gmail.com) + */ +public class S3UploadHandle implements Handle { + private Upload mUpload; + + public S3UploadHandle(Upload u) { + mUpload = u; + } + + public UploadResult get() throws Exception { + return mUpload.waitForUploadResult(); + } +} diff --git a/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java new file mode 100644 index 000000000..4d43d9f9d --- /dev/null +++ b/src/main/java/com/pinterest/secor/uploader/S3UploadManager.java @@ -0,0 +1,201 @@ +/** + * 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.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; +import com.amazonaws.services.s3.model.SSECustomerKey; +import com.pinterest.secor.common.*; +import com.pinterest.secor.util.FileUtil; +import com.amazonaws.ClientConfiguration; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.regions.Region; +import com.amazonaws.regions.Regions; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.AmazonS3Client; +import com.amazonaws.services.s3.S3ClientOptions; +import com.amazonaws.services.s3.transfer.Upload; +import com.amazonaws.services.s3.transfer.TransferManager; +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider; +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.text.SimpleDateFormat; +import java.util.Date; + +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.util.FileUtil; + +/** + * Manages uploads to S3 using the TransferManager class from the AWS + * SDK. + *

+ * Set the aws.sse.type property to specify the type of + * encryption to use. Supported options are: + * S3, KMS and customer. See AWS + * documentation for Server-Side Encryption (SSE) for details on these + * options.
+ * Leave blank to use unencrypted uploads.
+ * If set to KMS, the aws.sse.kms.key property + * specifies the id of the key to use. Leave unset to use the default AWS + * key.
+ * If set to customer, the aws.sse.customer.key + * property must be set to the base64 encoded customer key to use. + *

+ * + * @author Liam Stewart (liam.stewart@gmail.com) + */ +public class S3UploadManager extends UploadManager { + private static final Logger LOG = LoggerFactory.getLogger(S3UploadManager.class); + private static final String KMS = "KMS"; + private static final String S3 = "S3"; + private static final String CUSTOMER = "customer"; + + private final String s3Path; + + private TransferManager mManager; + + public S3UploadManager(SecorConfig config) { + super(config); + + final String accessKey = mConfig.getAwsAccessKey(); + final String secretKey = mConfig.getAwsSecretKey(); + final String endpoint = mConfig.getAwsEndpoint(); + final String region = mConfig.getAwsRegion(); + final String awsRole = mConfig.getAwsRole(); + + s3Path = mConfig.getS3Path(); + + AmazonS3 client; + AWSCredentialsProvider provider; + + ClientConfiguration clientConfiguration = new ClientConfiguration(); + boolean isHttpProxyEnabled = mConfig.getAwsProxyEnabled(); + + //proxy settings + if(isHttpProxyEnabled){ + LOG.info("Http Proxy Enabled for S3UploadManager"); + String httpProxyHost = mConfig.getAwsProxyHttpHost(); + int httpProxyPort = mConfig.getAwsProxyHttpPort(); + clientConfiguration.setProxyHost(httpProxyHost); + clientConfiguration.setProxyPort(httpProxyPort); + } + + if (accessKey.isEmpty() || secretKey.isEmpty()) { + provider = new DefaultAWSCredentialsProviderChain(); + } else { + provider = new AWSCredentialsProvider() { + public AWSCredentials getCredentials() { + return new BasicAWSCredentials(accessKey, secretKey); + } + public void refresh() {} + }; + } + + if (!awsRole.isEmpty()) { + provider = new STSAssumeRoleSessionCredentialsProvider(provider, awsRole, "secor"); + } + + client = new AmazonS3Client(provider, clientConfiguration); + + if (mConfig.getAwsClientPathStyleAccess()) { + S3ClientOptions clientOptions = new S3ClientOptions(); + clientOptions.setPathStyleAccess(true); + client.setS3ClientOptions(clientOptions); + } + + if (!endpoint.isEmpty()) { + client.setEndpoint(endpoint); + } else if (!region.isEmpty()) { + client.setRegion(Region.getRegion(Regions.fromName(region))); + } + + mManager = new TransferManager(client); + } + + public Handle upload(LogFilePath localPath) throws Exception { + String s3Bucket = mConfig.getS3Bucket(); + String curS3Path = s3Path; + String s3Key; + + File localFile = new File(localPath.getLogFilePath()); + + if (FileUtil.s3PathPrefixIsAltered(localPath.withPrefix(curS3Path).getLogFilePath(), mConfig)) { + curS3Path = FileUtil.getS3AlternativePathPrefix(mConfig); + LOG.info("Will upload file {} to alternative s3 path s3://{}/{}", localFile, s3Bucket, curS3Path); + } + + if (mConfig.getS3MD5HashPrefix()) { + // add MD5 hash to the prefix to have proper partitioning of the secor logs on s3 + String md5Hash = FileUtil.getMd5Hash(localPath.getTopic(), localPath.getPartitions()); + s3Key = localPath.withPrefix(md5Hash + "/" + curS3Path).getLogFilePath(); + } + else { + s3Key = localPath.withPrefix(curS3Path).getLogFilePath(); + } + + // make upload request, taking into account configured options for encryption + PutObjectRequest uploadRequest = new PutObjectRequest(s3Bucket, s3Key, localFile); + if (!mConfig.getAwsSseType().isEmpty()) { + if (S3.equals(mConfig.getAwsSseType())) { + LOG.info("uploading file {} to s3://{}/{} with S3-managed encryption", localFile, s3Bucket, s3Key); + enableS3Encryption(uploadRequest); + } else if (KMS.equals(mConfig.getAwsSseType())) { + LOG.info("uploading file {} to s3://{}/{} using KMS based encryption", localFile, s3Bucket, s3Key); + enableKmsEncryption(uploadRequest); + } else if (CUSTOMER.equals(mConfig.getAwsSseType())) { + LOG.info("uploading file {} to s3://{}/{} using customer key encryption", localFile, s3Bucket, s3Key); + enableCustomerEncryption(uploadRequest); + } else { + // bad option + throw new IllegalArgumentException(mConfig.getAwsSseType() + "is not a suitable type for AWS SSE encryption"); + } + } else { + LOG.info("uploading file {} to s3://{}/{} with no encryption", localFile, s3Bucket, s3Key); + } + + Upload upload = mManager.upload(uploadRequest); + return new S3UploadHandle(upload); + } + + private void enableCustomerEncryption(PutObjectRequest uploadRequest) { + SSECustomerKey sseKey = new SSECustomerKey(mConfig.getAwsSseCustomerKey()); + uploadRequest.withSSECustomerKey(sseKey); + } + + private void enableKmsEncryption(PutObjectRequest uploadRequest) { + String keyId = mConfig.getAwsSseKmsKey(); + if (!keyId.isEmpty()) { + uploadRequest.withSSEAwsKeyManagementParams(new SSEAwsKeyManagementParams(keyId)); + } else { + uploadRequest.withSSEAwsKeyManagementParams(new SSEAwsKeyManagementParams()); + } + } + + private void enableS3Encryption(PutObjectRequest uploadRequest) { + ObjectMetadata objectMetadata = new ObjectMetadata(); + objectMetadata.setSSEAlgorithm(ObjectMetadata.AES_256_SERVER_SIDE_ENCRYPTION); + uploadRequest.setMetadata(objectMetadata); + } +} diff --git a/src/main/java/com/pinterest/secor/uploader/UploadManager.java b/src/main/java/com/pinterest/secor/uploader/UploadManager.java new file mode 100644 index 000000000..8700cc399 --- /dev/null +++ b/src/main/java/com/pinterest/secor/uploader/UploadManager.java @@ -0,0 +1,34 @@ +/** + * 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.*; + +/** + * Manages uploads. + * + * @author Liam Stewart (liam.stewart@gmail.com) + */ +public abstract class UploadManager { + protected SecorConfig mConfig; + + public UploadManager(SecorConfig config) { + mConfig = config; + } + + public abstract Handle upload(LogFilePath localPath) throws Exception; +} diff --git a/src/main/java/com/pinterest/secor/uploader/Uploader.java b/src/main/java/com/pinterest/secor/uploader/Uploader.java index 0c12e4153..a9d1e6b3c 100644 --- a/src/main/java/com/pinterest/secor/uploader/Uploader.java +++ b/src/main/java/com/pinterest/secor/uploader/Uploader.java @@ -16,84 +16,105 @@ */ package com.pinterest.secor.uploader; +import com.google.common.base.Joiner; import com.pinterest.secor.common.*; -import com.pinterest.secor.common.SecorConfig; -import com.pinterest.secor.util.FileUtil; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileWriter; +import com.pinterest.secor.io.KeyValue; +import com.pinterest.secor.monitoring.MetricCollector; +import com.pinterest.secor.util.CompressionUtil; import com.pinterest.secor.util.IdUtil; import com.pinterest.secor.util.ReflectionUtil; -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.apache.commons.lang.StringUtils; import org.apache.hadoop.io.compress.CompressionCodec; +import org.joda.time.DateTime; 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. + * uploaded to the cloud. * * @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; + protected SecorConfig mConfig; + protected MetricCollector mMetricCollector; + protected OffsetTracker mOffsetTracker; + protected FileRegistry mFileRegistry; + protected ZookeeperConnector mZookeeperConnector; + protected UploadManager mUploadManager; + protected String mTopicFilter; + - public Uploader(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry) { - this(config, offsetTracker, fileRegistry, new ZookeeperConnector(config)); + /** + * Init the Uploader with its dependent objects. + * + * @param config Secor configuration + * @param offsetTracker Tracker of the current offset of topics partitions + * @param fileRegistry Registry of log files on a per-topic and per-partition basis + * @param uploadManager Manager of the physical upload of log files to the remote repository + * @param metricCollector component that ingest metrics into monitoring system + */ + public void init(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry, + UploadManager uploadManager, MetricCollector metricCollector) { + init(config, offsetTracker, fileRegistry, uploadManager, + new ZookeeperConnector(config), metricCollector); } // For testing use only. - public Uploader(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry, - ZookeeperConnector zookeeperConnector) { + public void init(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry, + UploadManager uploadManager, + ZookeeperConnector zookeeperConnector, MetricCollector metricCollector) { mConfig = config; mOffsetTracker = offsetTracker; mFileRegistry = fileRegistry; + mUploadManager = uploadManager; mZookeeperConnector = zookeeperConnector; + mTopicFilter = mConfig.getKafkaTopicUploadAtMinuteMarkFilter(); + mMetricCollector = metricCollector; } - 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(), - localPath.getExtension()); - String localLogFilename = localPath.getLogFilePath(); - LOG.info("uploading file " + localLogFilename + " to " + s3Path.getLogFilePath()); - FileUtil.moveToS3(localLogFilename, s3Path.getLogFilePath()); - } - - private void uploadFiles(TopicPartition topicPartition) throws Exception { + protected 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); + + String stripped = StringUtils.strip(mConfig.getZookeeperPath(), "/"); + final String lockPath = Joiner.on("/").skipNulls().join( + "", + stripped.isEmpty() ? null : stripped, + "secor", + "locks", + topicPartition.getTopic(), + topicPartition.getPartition()); + mZookeeperConnector.lock(lockPath); try { // Check if the committed offset has changed. - long zookeeperComittedOffsetCount = mZookeeperConnector.getCommittedOffsetCount( + long zookeeperCommittedOffsetCount = mZookeeperConnector.getCommittedOffsetCount( topicPartition); - if (zookeeperComittedOffsetCount == committedOffsetCount) { - LOG.info("uploading topic " + topicPartition.getTopic() + " partition " + - topicPartition.getPartition()); + if (zookeeperCommittedOffsetCount == committedOffsetCount) { + LOG.info("uploading topic {} partition {}", topicPartition.getTopic(), topicPartition.getPartition()); + // Deleting writers closes their streams flushing all pending data to the disk. + mFileRegistry.deleteWriters(topicPartition); Collection paths = mFileRegistry.getPaths(topicPartition); + List> uploadHandles = new ArrayList>(); for (LogFilePath path : paths) { - upload(path); + uploadHandles.add(mUploadManager.upload(path)); + } + for (Handle uploadHandle : uploadHandles) { + uploadHandle.get(); } mFileRegistry.deleteTopicPartition(topicPartition); mZookeeperConnector.setCommittedOffsetCount(topicPartition, lastSeenOffset + 1); mOffsetTracker.setCommittedOffsetCount(topicPartition, lastSeenOffset + 1); + + mMetricCollector.increment("uploader.file_uploads.count", paths.size(), topicPartition.getTopic()); } } finally { mZookeeperConnector.unlock(lockPath); @@ -102,47 +123,49 @@ private void uploadFiles(TopicPartition topicPartition) throws Exception { /** * This method is intended to be overwritten in tests. + * @throws Exception */ - protected SequenceFile.Reader createReader(FileSystem fileSystem, Path path, - Configuration configuration) throws IOException { - return new SequenceFile.Reader(fileSystem, path, configuration); + protected FileReader createReader(LogFilePath srcPath, CompressionCodec codec) throws Exception { + return ReflectionUtil.createFileReader( + mConfig.getFileReaderWriterFactory(), + srcPath, + codec, + mConfig + ); } 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; + FileReader reader = null; + FileWriter 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(); CompressionCodec codec = null; String extension = ""; if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { - codec = (CompressionCodec) ReflectionUtil.createCompressionCodec(mConfig.getCompressionCodec()); + codec = CompressionUtil.createCompressionCodec(mConfig.getCompressionCodec()); extension = codec.getDefaultExtension(); } - while (reader.next(key, value)) { - if (key.get() >= startOffset) { + reader = createReader(srcPath, codec); + KeyValue keyVal; + while ((keyVal = reader.next()) != null) { + if (keyVal.getOffset() >= startOffset) { if (writer == null) { String localPrefix = mConfig.getLocalPath() + '/' + IdUtil.getLocalMessageDir(); dstPath = new LogFilePath(localPrefix, srcPath.getTopic(), srcPath.getPartitions(), srcPath.getGeneration(), - srcPath.getKafkaPartition(), startOffset, extension); - writer = mFileRegistry.getOrCreateWriter(dstPath, codec); + srcPath.getKafkaPartition(), startOffset, + extension); + writer = mFileRegistry.getOrCreateWriter(dstPath, + codec); } - writer.append(key, value); + writer.write(keyVal); copiedMessages++; } } @@ -153,45 +176,63 @@ private void trim(LogFilePath srcPath, long startOffset) throws Exception { } mFileRegistry.deletePath(srcPath); if (dstPath == null) { - LOG.info("removed file " + srcPath.getLogFilePath()); + LOG.info("removed file {}", srcPath.getLogFilePath()); } else { - LOG.info("trimmed " + copiedMessages + " messages from " + srcFilename + " to " + - dstPath.getLogFilePath() + " with start offset " + startOffset); + LOG.info("trimmed {} messages from {} to {} with start offset {}", + copiedMessages, srcPath.getLogFilePath(), dstPath.getLogFilePath(), startOffset); } } - private void trimFiles(TopicPartition topicPartition, long startOffset) throws Exception { + protected 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 { + /*** + * If the topic is in the list of topics to upload at a specific time. For example at a minute mark. + * @param topicPartition + * @return + * @throws Exception + */ + private boolean isRequiredToUploadAtTime(TopicPartition topicPartition) throws Exception{ + final String topic = topicPartition.getTopic(); + if (mTopicFilter == null || mTopicFilter.isEmpty()){ + return false; + } + if (topic.matches(mTopicFilter)){ + if (DateTime.now().minuteOfHour().get() == mConfig.getUploadMinuteMark()){ + return true; + } + } + return false; + } + + protected void checkTopicPartition(TopicPartition topicPartition) throws Exception { final long size = mFileRegistry.getSize(topicPartition); final long modificationAgeSec = mFileRegistry.getModificationAgeSec(topicPartition); + LOG.debug("size: " + size + " modificationAge: " + modificationAgeSec); if (size >= mConfig.getMaxFileSizeBytes() || - modificationAgeSec >= mConfig.getMaxFileAgeSeconds()) { + modificationAgeSec >= mConfig.getMaxFileAgeSeconds() || + isRequiredToUploadAtTime(topicPartition)) { long newOffsetCount = mZookeeperConnector.getCommittedOffsetCount(topicPartition); long oldOffsetCount = mOffsetTracker.setCommittedOffsetCount(topicPartition, newOffsetCount); long lastSeenOffset = mOffsetTracker.getLastSeenOffset(topicPartition); if (oldOffsetCount == newOffsetCount) { + LOG.debug("Uploading for: " + topicPartition); 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()); + LOG.debug("last seen offset {} is lower than committed offset count {}. Deleting files in topic {} partition {}", + lastSeenOffset, newOffsetCount,topicPartition.getTopic(), 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()); + LOG.debug("previous committed offset count {} is lower than committed offset {} is lower than or equal to last seen offset {}. " + + "Trimming files in topic {} partition {}", + oldOffsetCount, newOffsetCount, lastSeenOffset, topicPartition.getTopic(), 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); @@ -199,10 +240,21 @@ private void checkTopicPartition(TopicPartition topicPartition) throws Exception } } + /** + * Apply the Uploader policy for pushing partition files to the underlying storage. + * + * For each of the partitions of the file registry, apply the policy for flushing + * them to the underlying storage. + * + * This method could be subclassed to provide an alternate policy. The custom uploader + * class name would need to be specified in the secor.upload.class. + * + * @throws Exception if any error occurs while appying the policy + */ public void applyPolicy() throws Exception { Collection topicPartitions = mFileRegistry.getTopicPartitions(); for (TopicPartition topicPartition : topicPartitions) { checkTopicPartition(topicPartition); } } -} +} \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/util/CompressionUtil.java b/src/main/java/com/pinterest/secor/util/CompressionUtil.java new file mode 100644 index 000000000..b988071f3 --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/CompressionUtil.java @@ -0,0 +1,43 @@ +/** + * 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.util.Collections; +import java.util.LinkedList; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.CompressionCodecFactory; + +/** + * + * Compression Codec related helper methods + * + * @author Praveen Murugesan (praveen@uber.com) + * + */ +public class CompressionUtil { + + public static CompressionCodec createCompressionCodec(String className) + throws Exception { + Configuration configuration = new Configuration(); + CompressionCodecFactory.setCodecClasses(configuration,new LinkedList(Collections.singletonList(Class.forName(className)))); + CompressionCodecFactory ccf = new CompressionCodecFactory( + configuration); + return ccf.getCodecByClassName(className); + } +} diff --git a/src/main/java/com/pinterest/secor/util/FileUtil.java b/src/main/java/com/pinterest/secor/util/FileUtil.java index 22ba33ed3..ad1fe6dac 100644 --- a/src/main/java/com/pinterest/secor/util/FileUtil.java +++ b/src/main/java/com/pinterest/secor/util/FileUtil.java @@ -16,16 +16,30 @@ */ package com.pinterest.secor.util; -import com.pinterest.secor.common.SecorConfig; +import java.io.File; +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.net.URI; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Date; + +import java.text.SimpleDateFormat; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.commons.lang3.StringUtils; 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.apache.hadoop.fs.s3a.Constants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.net.URI; -import java.util.ArrayList; -import java.util.Arrays; +import com.pinterest.secor.common.SecorConfig; /** * File util implements utilities for interactions with the file system. @@ -33,19 +47,102 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ public class FileUtil { - private static SecorConfig mConfig = null; + private static final Logger LOG = LoggerFactory.getLogger(FileUtil.class); + private static Configuration mConf = new Configuration(true); + private static final char[] m_digits = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', + 'b', 'c', 'd', 'e', 'f'}; + private static final Pattern datePattern = Pattern.compile(".*dt=(\\d\\d\\d\\d-\\d\\d-\\d\\d).*"); public static void configure(SecorConfig config) { - mConfig = config; + if (config != null) { + if (config.getCloudService().equals("Swift")) { + mConf.set("fs.swift.impl", "org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem"); + mConf.set("fs.swift.service.GENERICPROJECT.auth.url", config.getSwiftAuthUrl()); + mConf.set("fs.swift.service.GENERICPROJECT.username", config.getSwiftUsername()); + mConf.set("fs.swift.service.GENERICPROJECT.tenant", config.getSwiftTenant()); + mConf.set("fs.swift.service.GENERICPROJECT.http.port", config.getSwiftPort()); + mConf.set("fs.swift.service.GENERICPROJECT.use.get.auth", config.getSwiftGetAuth()); + mConf.set("fs.swift.service.GENERICPROJECT.public", config.getSwiftPublic()); + if (config.getSwiftGetAuth().equals("true")) { + mConf.set("fs.swift.service.GENERICPROJECT.apikey", config.getSwiftApiKey()); + } else { + mConf.set("fs.swift.service.GENERICPROJECT.password", config.getSwiftPassword()); + } + } else if (config.getCloudService().equals("S3")) { + if (config.getAwsAccessKey().isEmpty() != config.getAwsSecretKey().isEmpty()) { + throw new IllegalArgumentException( + "Must specify both aws.access.key and aws.secret.key or neither."); + } + if (!config.getAwsAccessKey().isEmpty()) { + mConf.set(Constants.ACCESS_KEY, config.getAwsAccessKey()); + mConf.set(Constants.SECRET_KEY, config.getAwsSecretKey()); + mConf.set("fs.s3n.awsAccessKeyId", config.getAwsAccessKey()); + mConf.set("fs.s3n.awsSecretAccessKey", config.getAwsSecretKey()); + } + } + } } 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), mConf); + } + + public static boolean s3PathPrefixIsAltered(String logFileName, SecorConfig config) + throws Exception { + Date logDate = null; + if (config.getS3AlterPathDate() != null && !config.getS3AlterPathDate().isEmpty()) { + + Date s3AlterPathDate = new SimpleDateFormat("yyyy-MM-dd").parse(config.getS3AlterPathDate()); + + // logFileName contains the log path, e.g. raw_logs/secor_topic/dt=2016-04-20/3_0_0000000000000292564 + Matcher dateMatcher = datePattern.matcher(logFileName); + if (dateMatcher.find()) { + logDate = new SimpleDateFormat("yyyy-MM-dd").parse(dateMatcher.group(1)); + } + + if (logDate == null) { + throw new Exception("Did not find a date in the format yyyy-MM-dd in " + logFileName); + } + + if (!s3AlterPathDate.after(logDate)) { + return true; + } } - return FileSystem.get(URI.create(path), conf); + + return false; + } + + public static String getS3AlternativePathPrefix(SecorConfig config) { + return config.getS3AlternativePath(); + } + + public static String getS3AlternativePrefix(SecorConfig config) { + return config.getS3FileSystem() + "://" + config.getS3Bucket() + "/" + config.getS3AlternativePath(); + } + + public static String getPrefix(String topic, SecorConfig config) throws IOException { + String prefix = null; + if (config.getCloudService().equals("Swift")) { + String container = null; + if (config.getSeparateContainersForTopics()) { + if (!exists("swift://" + topic + ".GENERICPROJECT")){ + String containerUrl = "swift://" + topic + ".GENERICPROJECT"; + Path containerPath = new Path(containerUrl); + getFileSystem(containerUrl).create(containerPath).close(); + } + container = topic; + } else { + container = config.getSwiftContainer(); + } + prefix = "swift://" + container + ".GENERICPROJECT/" + config.getSwiftPath(); + } else if (config.getCloudService().equals("S3")) { + prefix = config.getS3Prefix(); + } else if (config.getCloudService().equals("GS")) { + prefix = "gs://" + config.getGsBucket() + "/" + config.getGsPath(); + } else if (config.getCloudService().equals("Azure")) { + prefix = "azure://" + config.getAzureContainer() + "/" + config.getAzurePath(); + } + return prefix; } public static String[] list(String path) throws IOException { @@ -56,7 +153,8 @@ public static String[] list(String path) throws IOException { if (statuses != null) { for (FileStatus status : statuses) { Path statusPath = status.getPath(); - if (path.startsWith("s3://") || path.startsWith("s3n://")) { + if (path.startsWith("s3://") || path.startsWith("s3n://") || path.startsWith("s3a://") || + path.startsWith("swift://") || path.startsWith("gs://")) { paths.add(statusPath.toUri().toString()); } else { paths.add(statusPath.toUri().getPath()); @@ -97,10 +195,15 @@ public static void delete(String path) throws IOException { } } - public static void moveToS3(String srcLocalPath, String dstS3Path) throws IOException { + public static void deleteOnExit(String path) { + File file = new File(path); + file.deleteOnExit(); + } + + public static void moveToCloud(String srcLocalPath, String dstCloudPath) throws IOException { Path srcPath = new Path(srcLocalPath); - Path dstPath = new Path(dstS3Path); - getFileSystem(dstS3Path).moveFromLocalFile(srcPath, dstPath); + Path dstPath = new Path(dstCloudPath); + getFileSystem(dstCloudPath).moveFromLocalFile(srcPath, dstPath); } public static void touch(String path) throws IOException { @@ -119,7 +222,8 @@ public static long getModificationTimeMsRecursive(String path) throws IOExceptio for (FileStatus fileStatus : statuses) { Path statusPath = fileStatus.getPath(); String stringPath; - if (path.startsWith("s3://") || path.startsWith("s3n://")) { + if (path.startsWith("s3://") || path.startsWith("s3n://") || path.startsWith("s3a://") || + path.startsWith("swift://") || path.startsWith("gs://")) { stringPath = statusPath.toUri().toString(); } else { stringPath = statusPath.toUri().getPath(); @@ -132,4 +236,39 @@ public static long getModificationTimeMsRecursive(String path) throws IOExceptio } return modificationTime; } + + /** Generate MD5 hash of topic and partitions. And extract first 4 characters of the MD5 hash. + * @param topic + * @param partitions + * @return + */ + public static String getMd5Hash(String topic, String[] partitions) { + ArrayList elements = new ArrayList(); + elements.add(topic); + for (String partition : partitions) { + elements.add(partition); + } + String pathPrefix = StringUtils.join(elements, "/"); + try { + final MessageDigest messageDigest = MessageDigest.getInstance("MD5"); + byte[] md5Bytes = messageDigest.digest(pathPrefix.getBytes("UTF-8")); + return getHexEncode(md5Bytes).substring(0, 4); + } catch (NoSuchAlgorithmException e) { + LOG.error(e.getMessage()); + } catch (UnsupportedEncodingException e) { + LOG.error(e.getMessage()); + } + return ""; + } + + private static String getHexEncode(byte[] bytes) { + final char[] chars = new char[bytes.length * 2]; + for (int i = 0; i < bytes.length; ++i) { + final int cx = i * 2; + final byte b = bytes[i]; + chars[cx] = m_digits[(b & 0xf0) >> 4]; + chars[cx + 1] = m_digits[(b & 0x0f)]; + } + return new String(chars); + } } diff --git a/src/main/java/com/pinterest/secor/util/ParquetUtil.java b/src/main/java/com/pinterest/secor/util/ParquetUtil.java new file mode 100644 index 000000000..36df5342e --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/ParquetUtil.java @@ -0,0 +1,22 @@ +package com.pinterest.secor.util; + +import com.pinterest.secor.common.SecorConfig; +import org.apache.parquet.hadoop.ParquetWriter; + +public class ParquetUtil { + public static int getParquetBlockSize(SecorConfig config) { + return config.getInt("parquet.block.size", ParquetWriter.DEFAULT_BLOCK_SIZE); + } + + public static int getParquetPageSize(SecorConfig config) { + return config.getInt("parquet.page.size", ParquetWriter.DEFAULT_PAGE_SIZE); + } + + public static boolean getParquetEnableDictionary(SecorConfig config) { + return config.getBoolean("parquet.enable.dictionary", ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED); + } + + public static boolean getParquetValidation(SecorConfig config) { + return config.getBoolean("parquet.validation", ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED); + } +} diff --git a/src/main/java/com/pinterest/secor/util/ProtobufUtil.java b/src/main/java/com/pinterest/secor/util/ProtobufUtil.java new file mode 100644 index 000000000..b7572cb88 --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/ProtobufUtil.java @@ -0,0 +1,124 @@ +package com.pinterest.secor.util; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.protobuf.Message; +import com.pinterest.secor.common.SecorConfig; + +/** + * Various utilities for working with protocol buffer encoded messages. This + * utility will look for protobuf class in the configuration. It can be either + * per Kafka topic configuration, for example: + * + * secor.protobuf.message.class.<topic>=<protobuf class name> + * + * or, it can be global configuration for all topics (in case all the topics + * transfer the same message type): + * + * secor.protobuf.message.class.*=<protobuf class name> + * + * @author Michael Spector (spektom@gmail.com) + */ +public class ProtobufUtil { + + private static final Logger LOG = LoggerFactory.getLogger(ProtobufUtil.class); + + private boolean allTopics; + private Map> messageClassByTopic = new HashMap>(); + private Map messageParseMethodByTopic = new HashMap(); + private Class messageClassForAll; + private Method messageParseMethodForAll; + + /** + * Creates new instance of {@link ProtobufUtil} + * + * @param config + * Secor configuration instance + * @throws RuntimeException + * when configuration option + * secor.protobuf.message.class is invalid. + */ + @SuppressWarnings("unchecked") + public ProtobufUtil(SecorConfig config) { + Map messageClassPerTopic = config.getProtobufMessageClassPerTopic(); + for (Entry entry : messageClassPerTopic.entrySet()) { + try { + String topic = entry.getKey(); + Class messageClass = (Class) Class.forName(entry.getValue()); + Method messageParseMethod = messageClass.getDeclaredMethod("parseFrom", + new Class[] { byte[].class }); + + allTopics = "*".equals(topic); + + if (allTopics) { + messageClassForAll = messageClass; + messageParseMethodForAll = messageParseMethod; + LOG.info("Using protobuf message class: {} for all Kafka topics", messageClass.getName()); + } else { + messageClassByTopic.put(topic, messageClass); + messageParseMethodByTopic.put(topic, messageParseMethod); + LOG.info("Using protobuf message class: {} for Kafka topic: {}", messageClass.getName(), topic); + } + } catch (ClassNotFoundException e) { + LOG.error("Unable to load protobuf message class", e); + } catch (NoSuchMethodException e) { + LOG.error("Unable to find parseFrom() method in protobuf message class", e); + } catch (SecurityException e) { + LOG.error("Unable to use parseFrom() method from protobuf message class", e); + } + } + } + + /** + * Returns whether there was a protobuf class configuration + */ + public boolean isConfigured() { + return allTopics || !messageClassByTopic.isEmpty(); + } + + /** + * Returns configured protobuf message class for the given Kafka topic + * + * @param topic + * Kafka topic + * @return protobuf message class used by this utility instance, or + * null in case valid class couldn't be found in the + * configuration. + */ + public Class getMessageClass(String topic) { + return allTopics ? messageClassForAll : messageClassByTopic.get(topic); + } + + /** + * Decodes protobuf message + * + * @param topic + * Kafka topic name + * @param payload + * Byte array containing encoded protobuf message + * @return protobuf message instance + * @throws RuntimeException + * when there's problem decoding protobuf message + */ + public Message decodeMessage(String topic, byte[] payload) { + try { + Method parseMethod = allTopics ? messageParseMethodForAll : messageParseMethodByTopic.get(topic); + return (Message) parseMethod.invoke(null, payload); + } catch (IllegalArgumentException e) { + throw new RuntimeException("Can't parse protobuf message, since parseMethod() is not callable. " + + "Please check your protobuf version (this code works with protobuf >= 2.6.1)", e); + } catch (IllegalAccessException e) { + throw new RuntimeException("Can't parse protobuf message, since parseMethod() is not accessible. " + + "Please check your protobuf version (this code works with protobuf >= 2.6.1)", e); + } catch (InvocationTargetException e) { + throw new RuntimeException("Error parsing protobuf message", e); + } + } +} diff --git a/src/main/java/com/pinterest/secor/util/RateLimitUtil.java b/src/main/java/com/pinterest/secor/util/RateLimitUtil.java index 57cb4b5c9..2e130fde2 100644 --- a/src/main/java/com/pinterest/secor/util/RateLimitUtil.java +++ b/src/main/java/com/pinterest/secor/util/RateLimitUtil.java @@ -35,7 +35,7 @@ public static void configure(SecorConfig config) { mRateLimiter = RateLimiter.create(config.getMessagesPerSecond()); } - public static void acquire() { - mRateLimiter.acquire(); + public static void acquire(int n) { + mRateLimiter.acquire(n); } } diff --git a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java index 8cad96d4a..8bcfbd787 100644 --- a/src/main/java/com/pinterest/secor/util/ReflectionUtil.java +++ b/src/main/java/com/pinterest/secor/util/ReflectionUtil.java @@ -5,9 +5,9 @@ * 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 - * + *

+ * 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. @@ -16,35 +16,237 @@ */ package com.pinterest.secor.util; +import com.pinterest.secor.common.LogFilePath; import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileReaderWriterFactory; +import com.pinterest.secor.io.FileWriter; +import com.pinterest.secor.monitoring.MetricCollector; +import com.pinterest.secor.parser.MessageParser; +import com.pinterest.secor.transformer.MessageTransformer; +import com.pinterest.secor.uploader.UploadManager; +import com.pinterest.secor.uploader.Uploader; +import com.pinterest.secor.util.orc.schema.ORCScehmaProvider; -import java.lang.reflect.Constructor; +import org.apache.hadoop.io.compress.CompressionCodec; /** - * ReflectionUtil implements utility methods to construct objects of classes specified by name. + * ReflectionUtil implements utility methods to construct objects of classes + * specified by name. * * @author Pawel Garbacki (pawel@pinterest.com) + * @author Silas Davis (github-code@silasdavis.net) */ public class ReflectionUtil { - public static Object createCompressionCodec(String className) throws Exception { + /** + * Create an UploadManager from its fully qualified class name. + * + * The class passed in by name must be assignable to UploadManager + * and have 1-parameter constructor accepting a SecorConfig. + * + * See the secor.upload.manager.class config option. + * + * @param className The class name of a subclass of UploadManager + * @param config The SecorCondig to initialize the UploadManager with + * @return an UploadManager instance with the runtime type of the class passed by name + * @throws Exception + */ + public static UploadManager createUploadManager(String className, + SecorConfig config) throws Exception { + Class clazz = Class.forName(className); + if (!UploadManager.class.isAssignableFrom(clazz)) { + throw new IllegalArgumentException(String.format("The class '%s' is not assignable to '%s'.", + className, UploadManager.class.getName())); + } + + // Assume that subclass of UploadManager has a constructor with the same signature as UploadManager + return (UploadManager) clazz.getConstructor(SecorConfig.class).newInstance(config); + } + + /** + * Create an Uploader from its fully qualified class name. + * + * The class passed in by name must be assignable to Uploader. + * See the secor.upload.class config option. + * + * @param className The class name of a subclass of Uploader + * @return an UploadManager instance with the runtime type of the class passed by name + * @throws Exception + */ + public static Uploader createUploader(String className) throws Exception { + Class clazz = Class.forName(className); + if (!Uploader.class.isAssignableFrom(clazz)) { + throw new IllegalArgumentException(String.format("The class '%s' is not assignable to '%s'.", + className, Uploader.class.getName())); + } + + return (Uploader) clazz.newInstance(); + } + + /** + * Create a MessageParser from it's fully qualified class name. + * The class passed in by name must be assignable to MessageParser and have 1-parameter constructor accepting a SecorConfig. + * Allows the MessageParser to be pluggable by providing the class name of a desired MessageParser in config. + * + * See the secor.message.parser.class config option. + * + * @param className The class name of a subclass of MessageParser + * @param config The SecorCondig to initialize the MessageParser with + * @return a MessageParser instance with the runtime type of the class passed by name + * @throws Exception + */ + public static MessageParser createMessageParser(String className, + SecorConfig config) throws Exception { + Class clazz = Class.forName(className); + if (!MessageParser.class.isAssignableFrom(clazz)) { + throw new IllegalArgumentException(String.format("The class '%s' is not assignable to '%s'.", + className, MessageParser.class.getName())); + } + + // Assume that subclass of MessageParser has a constructor with the same signature as MessageParser + return (MessageParser) clazz.getConstructor(SecorConfig.class).newInstance(config); + } + + /** + * Create a FileReaderWriterFactory that is able to read and write a specific type of output log file. + * The class passed in by name must be assignable to FileReaderWriterFactory. + * Allows for pluggable FileReader and FileWriter instances to be constructed for a particular type of log file. + * + * See the secor.file.reader.writer.factory config option. + * + * @param className the class name of a subclass of FileReaderWriterFactory + * @param config The SecorCondig to initialize the FileReaderWriterFactory with + * @return a FileReaderWriterFactory with the runtime type of the class passed by name + * @throws Exception + */ + private static FileReaderWriterFactory createFileReaderWriterFactory(String className, + SecorConfig config) throws Exception { Class clazz = Class.forName(className); - return clazz.getConstructor().newInstance(); + if (!FileReaderWriterFactory.class.isAssignableFrom(clazz)) { + throw new IllegalArgumentException(String.format("The class '%s' is not assignable to '%s'.", + className, FileReaderWriterFactory.class.getName())); + } + + try { + // Try to load constructor that accepts single parameter - secor + // configuration instance + return (FileReaderWriterFactory) clazz.getConstructor(SecorConfig.class).newInstance(config); + } catch (NoSuchMethodException e) { + // Fallback to parameterless constructor + return (FileReaderWriterFactory) clazz.newInstance(); + } + } + + /** + * Use the FileReaderWriterFactory specified by className to build a FileWriter + * + * @param className the class name of a subclass of FileReaderWriterFactory to create a FileWriter from + * @param logFilePath the LogFilePath that the returned FileWriter should write to + * @param codec an instance CompressionCodec to compress the file written with, or null for no compression + * @param config The SecorCondig to initialize the FileWriter with + * @return a FileWriter specialised to write the type of files supported by the FileReaderWriterFactory + * @throws Exception + */ + public static FileWriter createFileWriter(String className, LogFilePath logFilePath, + CompressionCodec codec, + SecorConfig config) + throws Exception { + return createFileReaderWriterFactory(className, config).BuildFileWriter(logFilePath, codec); } - public static Object createMessageParser(String className, - SecorConfig config) throws Exception { + /** + * Use the FileReaderWriterFactory specified by className to build a FileReader + * + * @param className the class name of a subclass of FileReaderWriterFactory to create a FileReader from + * @param logFilePath the LogFilePath that the returned FileReader should read from + * @param codec an instance CompressionCodec to decompress the file being read, or null for no compression + * @param config The SecorCondig to initialize the FileReader with + * @return a FileReader specialised to read the type of files supported by the FileReaderWriterFactory + * @throws Exception + */ + public static FileReader createFileReader(String className, LogFilePath logFilePath, + CompressionCodec codec, + SecorConfig config) + throws Exception { + return createFileReaderWriterFactory(className, config).BuildFileReader(logFilePath, codec); + } + + /** + * Create a MessageTransformer from it's fully qualified class name. The + * class passed in by name must be assignable to MessageTransformers and have + * 1-parameter constructor accepting a SecorConfig. Allows the MessageTransformers + * to be pluggable by providing the class name of a desired MessageTransformers in + * config. + * + * See the secor.message.transformer.class config option. + * + * @param className + * @param config + * @return + * @throws Exception + */ + public static MessageTransformer createMessageTransformer( + String className, SecorConfig config) throws Exception { Class clazz = Class.forName(className); + if (!MessageTransformer.class.isAssignableFrom(clazz)) { + throw new IllegalArgumentException(String.format( + "The class '%s' is not assignable to '%s'.", className, + MessageTransformer.class.getName())); + } + return (MessageTransformer) clazz.getConstructor(SecorConfig.class) + .newInstance(config); + } - // Search for an "appropriate" constructor. - for (Constructor ctor : clazz.getConstructors()) { - Class[] paramTypes = ctor.getParameterTypes(); + /** + * Create an MetricCollector from its fully qualified class name. + *

+ * The class passed in by name must be assignable to MetricCollector. + * See the secor.monitoring.metrics.collector.class config option. + * + * @param className The class name of a subclass of MetricCollector + * @return a MetricCollector with the runtime type of the class passed by name + * @throws ClassNotFoundException if class with the {@code className} is not found in classpath + * @throws IllegalAccessException if the class or its nullary + * constructor is not accessible. + * @throws InstantiationException if this {@code Class} represents an abstract class, + * an interface, an array class, a primitive type, or void; + * or if the class has no nullary constructor; + * or if the instantiation fails for some other reason. + */ + public static MetricCollector createMetricCollector(String className) + throws ClassNotFoundException, IllegalAccessException, InstantiationException { + Class clazz = Class.forName(className); + if (!MetricCollector.class.isAssignableFrom(clazz)) { + throw new IllegalArgumentException(String.format("The class '%s' is not assignable to '%s'.", + className, MetricCollector.class.getName())); + } - // If the arity matches, let's use it. - if (paramTypes.length == 1) { - Object[] args = {config}; - return ctor.newInstance(args); - } + return (MetricCollector) clazz.newInstance(); + } + + /** + * Create a ORCScehmaProvider from it's fully qualified class name. The + * class passed in by name must be assignable to ORCScehmaProvider and have + * 1-parameter constructor accepting a SecorConfig. Allows the ORCScehmaProvider + * to be pluggable by providing the class name of a desired ORCScehmaProvider in + * config. + * + * See the secor.orc.schema.provider config option. + * + * @param className + * @param config + * @return + * @throws Exception + */ + public static ORCScehmaProvider createORCSchemaProvider( + String className, SecorConfig config) throws Exception { + Class clazz = Class.forName(className); + if (!ORCScehmaProvider.class.isAssignableFrom(clazz)) { + throw new IllegalArgumentException(String.format( + "The class '%s' is not assignable to '%s'.", className, + ORCScehmaProvider.class.getName())); } - throw new IllegalArgumentException("Class not found " + className); + return (ORCScehmaProvider) clazz.getConstructor(SecorConfig.class) + .newInstance(config); } } diff --git a/src/main/java/com/pinterest/secor/util/StatsUtil.java b/src/main/java/com/pinterest/secor/util/StatsUtil.java index 9cb924dad..79f1920fb 100644 --- a/src/main/java/com/pinterest/secor/util/StatsUtil.java +++ b/src/main/java/com/pinterest/secor/util/StatsUtil.java @@ -35,4 +35,9 @@ public static void clearLabel(String name) { name += "." + threadId; Stats.clearLabel(name); } + + public static void incr(String name) { + Stats.incr(name); + } + } diff --git a/src/main/java/com/pinterest/secor/util/ThriftUtil.java b/src/main/java/com/pinterest/secor/util/ThriftUtil.java new file mode 100644 index 000000000..75a1d1a53 --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/ThriftUtil.java @@ -0,0 +1,122 @@ +package com.pinterest.secor.util; + +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.thrift.TBase; +import org.apache.thrift.TDeserializer; +import org.apache.thrift.TException; +import org.apache.thrift.TSerializer; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TProtocolFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.pinterest.secor.common.SecorConfig; + +/** + * Adapted from ProtobufUtil Various utilities for working with thrift encoded + * messages. This utility will look for thrift class in the configuration. It + * can be either per Kafka topic configuration, for example: + * + * secor.thrift.message.class.<topic>=<thrift class name> + * + * or, it can be global configuration for all topics (in case all the topics + * transfer the same message type): + * + * secor.thrift.message.class.*=<thrift class name> + * + * @author jaime sastre (jaime sastre.s@gmail.com) + */ +public class ThriftUtil { + + private static final Logger LOG = LoggerFactory.getLogger(ThriftUtil.class); + + private boolean allTopics; + @SuppressWarnings("rawtypes") + private Map> messageClassByTopic = new HashMap>(); + @SuppressWarnings("rawtypes") + private Class messageClassForAll; + private TProtocolFactory messageProtocolFactory; + + /** + * Creates new instance of {@link ThriftUtil} + * + * @param config + * Secor configuration instance + * @throws RuntimeException + * when configuration option + * secor.thrift.message.class is invalid. + */ + @SuppressWarnings({ "rawtypes", "unchecked" }) + public ThriftUtil(SecorConfig config) { + Map messageClassPerTopic = config.getThriftMessageClassPerTopic(); + + for (Entry entry : messageClassPerTopic.entrySet()) { + try { + String topic = entry.getKey(); + Class messageClass = (Class) Class.forName(entry.getValue()); + + allTopics = "*".equals(topic); + + if (allTopics) { + messageClassForAll = messageClass; + LOG.info("Using thrift message class: {} for all Kafka topics", messageClass.getName()); + } else { + messageClassByTopic.put(topic, messageClass); + LOG.info("Using thrift message class: {} for Kafka topic: {}", messageClass.getName(), topic); + } + } catch (ClassNotFoundException e) { + LOG.error("Unable to load thrift message class", e); + } + } + + try { + String protocolName = config.getThriftProtocolClass(); + + if (protocolName != null) { + String factoryClassName = protocolName.concat("$Factory"); + messageProtocolFactory = ((Class) Class.forName(factoryClassName)).newInstance(); + } else + messageProtocolFactory = new TBinaryProtocol.Factory(); + + } catch (ClassNotFoundException e) { + LOG.error("Unable to load thrift protocol class", e); + } catch (InstantiationException e) { + LOG.error("Unable to load thrift protocol class", e); + } catch (IllegalAccessException e) { + LOG.error("Unable to load thrift protocol class", e); + } + } + + /** + * Returns configured thrift message class for the given Kafka topic + * + * @param topic + * Kafka topic + * @return thrift message class used by this utility instance, or + * null in case valid class couldn't be found in the + * configuration. + */ + @SuppressWarnings("rawtypes") + public Class getMessageClass(String topic) { + return allTopics ? messageClassForAll : messageClassByTopic.get(topic); + } + + @SuppressWarnings("rawtypes") + public TBase decodeMessage(String topic, byte[] payload) + throws InstantiationException, IllegalAccessException, TException { + TDeserializer serializer = new TDeserializer(messageProtocolFactory); + TBase result = this.getMessageClass(topic).newInstance(); + serializer.deserialize(result, payload); + return result; + } + + @SuppressWarnings("rawtypes") + public byte[] encodeMessage(TBase object) throws InstantiationException, + IllegalAccessException, TException { + TSerializer serializer = new TSerializer(messageProtocolFactory); + return serializer.serialize(object); + } +} diff --git a/src/main/java/com/pinterest/secor/util/orc/JsonFieldFiller.java b/src/main/java/com/pinterest/secor/util/orc/JsonFieldFiller.java new file mode 100644 index 000000000..865af74a6 --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/orc/JsonFieldFiller.java @@ -0,0 +1,126 @@ +package com.pinterest.secor.util.orc; + +import java.util.List; + +import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.orc.TypeDescription; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONWriter; + +/** + * + * @author Ashish (ashu.impetus@gmail.com) + * + */ +public class JsonFieldFiller { + + public static void processRow(JSONWriter writer, VectorizedRowBatch batch, + TypeDescription schema, int row) throws JSONException { + if (schema.getCategory() == TypeDescription.Category.STRUCT) { + List fieldTypes = schema.getChildren(); + List fieldNames = schema.getFieldNames(); + writer.object(); + for (int c = 0; c < batch.cols.length; ++c) { + writer.key(fieldNames.get(c)); + setValue(writer, batch.cols[c], fieldTypes.get(c), row); + } + writer.endObject(); + } else { + setValue(writer, batch.cols[0], schema, row); + } + } + + static void setValue(JSONWriter writer, ColumnVector vector, + TypeDescription schema, int row) throws JSONException { + if (vector.isRepeating) { + row = 0; + } + if (vector.noNulls || !vector.isNull[row]) { + switch (schema.getCategory()) { + case BOOLEAN: + writer.value(((LongColumnVector) vector).vector[row] != 0); + break; + case BYTE: + case SHORT: + case INT: + case LONG: + writer.value(((LongColumnVector) vector).vector[row]); + break; + case FLOAT: + case DOUBLE: + writer.value(((DoubleColumnVector) vector).vector[row]); + break; + case STRING: + case CHAR: + case VARCHAR: + writer.value(((BytesColumnVector) vector).toString(row)); + break; + case DECIMAL: + writer.value(((DecimalColumnVector) vector).vector[row] + .toString()); + break; + case DATE: + writer.value(new DateWritable( + (int) ((LongColumnVector) vector).vector[row]) + .toString()); + break; + case TIMESTAMP: + writer.value(((TimestampColumnVector) vector) + .asScratchTimestamp(row).toString()); + break; + case LIST: + setList(writer, (ListColumnVector) vector, schema, row); + break; + case STRUCT: + setStruct(writer, (StructColumnVector) vector, schema, row); + break; + case UNION: + // printUnion(writer, (UnionColumnVector) vector, schema, row); + break; + case BINARY: + // printBinary(writer, (BytesColumnVector) vector, row); + break; + case MAP: + // printMap(writer, (MapColumnVector) vector, schema, row); + break; + default: + throw new IllegalArgumentException("Unknown type " + + schema.toString()); + } + } else { + writer.value(null); + } + } + + private static void setList(JSONWriter writer, ListColumnVector vector, + TypeDescription schema, int row) throws JSONException { + writer.array(); + int offset = (int) vector.offsets[row]; + TypeDescription childType = schema.getChildren().get(0); + for (int i = 0; i < vector.lengths[row]; ++i) { + setValue(writer, vector.child, childType, offset + i); + } + writer.endArray(); + } + + private static void setStruct(JSONWriter writer, StructColumnVector batch, + TypeDescription schema, int row) throws JSONException { + writer.object(); + List fieldNames = schema.getFieldNames(); + List fieldTypes = schema.getChildren(); + for (int i = 0; i < fieldTypes.size(); ++i) { + writer.key(fieldNames.get(i)); + setValue(writer, batch.fields[i], fieldTypes.get(i), row); + } + writer.endObject(); + } +} diff --git a/src/main/java/com/pinterest/secor/util/orc/VectorColumnFiller.java b/src/main/java/com/pinterest/secor/util/orc/VectorColumnFiller.java new file mode 100644 index 000000000..73284f496 --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/orc/VectorColumnFiller.java @@ -0,0 +1,227 @@ +package com.pinterest.secor.util.orc; + +import java.nio.charset.StandardCharsets; +import java.sql.Timestamp; +import java.util.List; + +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.orc.TypeDescription; + +import com.google.gson.JsonArray; +import com.google.gson.JsonElement; +import com.google.gson.JsonObject; + +/** + * + * @author Ashish (ashu.impetus@gmail.com) + * + */ +public class VectorColumnFiller { + + public interface JsonConverter { + void convert(JsonElement value, ColumnVector vect, int row); + } + + static class BooleanColumnConverter implements JsonConverter { + public void convert(JsonElement value, ColumnVector vect, int row) { + if (value == null || value.isJsonNull()) { + vect.noNulls = false; + vect.isNull[row] = true; + } else { + LongColumnVector vector = (LongColumnVector) vect; + vector.vector[row] = value.getAsBoolean() ? 1 : 0; + } + } + } + + static class LongColumnConverter implements JsonConverter { + public void convert(JsonElement value, ColumnVector vect, int row) { + if (value == null || value.isJsonNull()) { + vect.noNulls = false; + vect.isNull[row] = true; + } else { + LongColumnVector vector = (LongColumnVector) vect; + vector.vector[row] = value.getAsLong(); + } + } + } + + static class DoubleColumnConverter implements JsonConverter { + public void convert(JsonElement value, ColumnVector vect, int row) { + if (value == null || value.isJsonNull()) { + vect.noNulls = false; + vect.isNull[row] = true; + } else { + DoubleColumnVector vector = (DoubleColumnVector) vect; + vector.vector[row] = value.getAsDouble(); + } + } + } + + static class StringColumnConverter implements JsonConverter { + public void convert(JsonElement value, ColumnVector vect, int row) { + if (value == null || value.isJsonNull()) { + vect.noNulls = false; + vect.isNull[row] = true; + } else { + BytesColumnVector vector = (BytesColumnVector) vect; + byte[] bytes = value.getAsString().getBytes( + StandardCharsets.UTF_8); + vector.setRef(row, bytes, 0, bytes.length); + } + } + } + + static class BinaryColumnConverter implements JsonConverter { + public void convert(JsonElement value, ColumnVector vect, int row) { + if (value == null || value.isJsonNull()) { + vect.noNulls = false; + vect.isNull[row] = true; + } else { + BytesColumnVector vector = (BytesColumnVector) vect; + String binStr = value.getAsString(); + byte[] bytes = new byte[binStr.length() / 2]; + for (int i = 0; i < bytes.length; ++i) { + bytes[i] = (byte) Integer.parseInt( + binStr.substring(i * 2, i * 2 + 2), 16); + } + vector.setRef(row, bytes, 0, bytes.length); + } + } + } + + static class TimestampColumnConverter implements JsonConverter { + public void convert(JsonElement value, ColumnVector vect, int row) { + if (value == null || value.isJsonNull()) { + vect.noNulls = false; + vect.isNull[row] = true; + } else { + TimestampColumnVector vector = (TimestampColumnVector) vect; + vector.set( + row, + Timestamp.valueOf(value.getAsString().replaceAll( + "[TZ]", " "))); + } + } + } + + static class DecimalColumnConverter implements JsonConverter { + public void convert(JsonElement value, ColumnVector vect, int row) { + if (value == null || value.isJsonNull()) { + vect.noNulls = false; + vect.isNull[row] = true; + } else { + DecimalColumnVector vector = (DecimalColumnVector) vect; + vector.vector[row].set(HiveDecimal.create(value.getAsString())); + } + } + } + + static class StructColumnConverter implements JsonConverter { + private JsonConverter[] childrenConverters; + private List fieldNames; + + public StructColumnConverter(TypeDescription schema) { + List kids = schema.getChildren(); + childrenConverters = new JsonConverter[kids.size()]; + for (int c = 0; c < childrenConverters.length; ++c) { + childrenConverters[c] = createConverter(kids.get(c)); + } + fieldNames = schema.getFieldNames(); + } + + public void convert(JsonElement value, ColumnVector vect, int row) { + if (value == null || value.isJsonNull()) { + vect.noNulls = false; + vect.isNull[row] = true; + } else { + StructColumnVector vector = (StructColumnVector) vect; + JsonObject obj = value.getAsJsonObject(); + for (int c = 0; c < childrenConverters.length; ++c) { + JsonElement elem = obj.get(fieldNames.get(c)); + childrenConverters[c].convert(elem, vector.fields[c], row); + } + } + } + } + + static class ListColumnConverter implements JsonConverter { + private JsonConverter childrenConverter; + + public ListColumnConverter(TypeDescription schema) { + childrenConverter = createConverter(schema.getChildren().get(0)); + } + + public void convert(JsonElement value, ColumnVector vect, int row) { + if (value == null || value.isJsonNull()) { + vect.noNulls = false; + vect.isNull[row] = true; + } else { + ListColumnVector vector = (ListColumnVector) vect; + JsonArray obj = value.getAsJsonArray(); + vector.lengths[row] = obj.size(); + vector.offsets[row] = vector.childCount; + vector.childCount += vector.lengths[row]; + vector.child.ensureSize(vector.childCount, true); + for (int c = 0; c < obj.size(); ++c) { + childrenConverter.convert(obj.get(c), vector.child, + (int) vector.offsets[row] + c); + } + } + } + } + + public static JsonConverter createConverter(TypeDescription schema) { + switch (schema.getCategory()) { + case BYTE: + case SHORT: + case INT: + case LONG: + return new LongColumnConverter(); + case FLOAT: + case DOUBLE: + return new DoubleColumnConverter(); + case CHAR: + case VARCHAR: + case STRING: + return new StringColumnConverter(); + case DECIMAL: + return new DecimalColumnConverter(); + case TIMESTAMP: + return new TimestampColumnConverter(); + case BINARY: + return new BinaryColumnConverter(); + case BOOLEAN: + return new BooleanColumnConverter(); + case STRUCT: + return new StructColumnConverter(schema); + case LIST: + return new ListColumnConverter(schema); + default: + throw new IllegalArgumentException("Unhandled type " + schema); + } + } + + public static void fillRow(int rowIndex, JsonConverter[] converters, + TypeDescription schema, VectorizedRowBatch batch, JsonObject data) { + List fieldNames = schema.getFieldNames(); + for (int c = 0; c < converters.length; ++c) { + JsonElement field = data.get(fieldNames.get(c)); + if (field == null) { + batch.cols[c].noNulls = false; + batch.cols[c].isNull[0] = true; + } else { + converters[c].convert(field, batch.cols[c], rowIndex); + } + } + } +} \ No newline at end of file diff --git a/src/main/java/com/pinterest/secor/util/orc/schema/DefaultORCSchemaProvider.java b/src/main/java/com/pinterest/secor/util/orc/schema/DefaultORCSchemaProvider.java new file mode 100644 index 000000000..7696725c8 --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/orc/schema/DefaultORCSchemaProvider.java @@ -0,0 +1,57 @@ +package com.pinterest.secor.util.orc.schema; + +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.orc.TypeDescription; + +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.common.SecorConfig; + +/** + * Default implementation for ORC schema provider. It fetches ORC schemas from + * configuration. User has to specify one schema per kafka topic or can have + * same schema for all the topics. + * + * @author Ashish (ashu.impetus@gmail.com) + * + */ +public class DefaultORCSchemaProvider implements ORCScehmaProvider { + + private Map topicToSchemaMap; + private TypeDescription schemaForAlltopic; + + public DefaultORCSchemaProvider(SecorConfig config) { + topicToSchemaMap = new HashMap(); + setSchemas(config); + } + + @Override + public TypeDescription getSchema(String topic, LogFilePath logFilePath) { + TypeDescription topicSpecificTD = topicToSchemaMap.get(topic); + if (null != topicSpecificTD) { + return topicSpecificTD; + } + return schemaForAlltopic; + } + + /** + * This method is used for fetching all ORC schemas from config + * + * @param config + */ + private void setSchemas(SecorConfig config) { + Map schemaPerTopic = config.getORCMessageSchema(); + for (Entry entry : schemaPerTopic.entrySet()) { + String topic = entry.getKey(); + TypeDescription schema = TypeDescription.fromString(entry + .getValue()); + topicToSchemaMap.put(topic, schema); + // If common schema is given + if ("*".equals(topic)) { + schemaForAlltopic = schema; + } + } + } +} diff --git a/src/main/java/com/pinterest/secor/util/orc/schema/ORCScehmaProvider.java b/src/main/java/com/pinterest/secor/util/orc/schema/ORCScehmaProvider.java new file mode 100644 index 000000000..cd96d1c07 --- /dev/null +++ b/src/main/java/com/pinterest/secor/util/orc/schema/ORCScehmaProvider.java @@ -0,0 +1,25 @@ +package com.pinterest.secor.util.orc.schema; + +import org.apache.orc.TypeDescription; + +import com.pinterest.secor.common.LogFilePath; + +/** + * ORC schema provider interface + * + * @author Ashish (ashu.impetus@gmail.com) + * + */ +public interface ORCScehmaProvider { + + /** + * This implementation should take a kafka topic name and returns ORC + * schema. ORC schema should be in the form of TypeDescription + * + * @param topic kafka topic + * @param logFilePath It may require to figure out the schema + * @return + */ + public TypeDescription getSchema(String topic, LogFilePath logFilePath); + +} diff --git a/src/main/java/com/pinterest/secor/writer/MessageWriter.java b/src/main/java/com/pinterest/secor/writer/MessageWriter.java index f757365fb..090c5a544 100644 --- a/src/main/java/com/pinterest/secor/writer/MessageWriter.java +++ b/src/main/java/com/pinterest/secor/writer/MessageWriter.java @@ -16,21 +16,24 @@ */ package com.pinterest.secor.writer; -import com.pinterest.secor.common.*; +import com.pinterest.secor.common.FileRegistry; +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.common.OffsetTracker; import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.common.TopicPartition; +import com.pinterest.secor.io.FileWriter; +import com.pinterest.secor.io.KeyValue; +import com.pinterest.secor.message.Message; import com.pinterest.secor.message.ParsedMessage; - -import java.io.IOException; - +import com.pinterest.secor.util.CompressionUtil; import com.pinterest.secor.util.IdUtil; -import com.pinterest.secor.util.ReflectionUtil; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.SequenceFile; +import com.pinterest.secor.util.StatsUtil; import org.apache.hadoop.io.compress.CompressionCodec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; + /** * Message writer appends Kafka messages to local log files. * @@ -39,11 +42,13 @@ public class MessageWriter { private static final Logger LOG = LoggerFactory.getLogger(MessageWriter.class); - private SecorConfig mConfig; - private OffsetTracker mOffsetTracker; - private FileRegistry mFileRegistry; - private String mFileExtension; - private CompressionCodec mCodec; + protected SecorConfig mConfig; + protected OffsetTracker mOffsetTracker; + protected FileRegistry mFileRegistry; + protected String mFileExtension; + protected CompressionCodec mCodec; + protected String mLocalPrefix; + protected final int mGeneration; public MessageWriter(SecorConfig config, OffsetTracker offsetTracker, FileRegistry fileRegistry) throws Exception { @@ -51,42 +56,44 @@ public MessageWriter(SecorConfig config, OffsetTracker offsetTracker, mOffsetTracker = offsetTracker; mFileRegistry = fileRegistry; if (mConfig.getCompressionCodec() != null && !mConfig.getCompressionCodec().isEmpty()) { - mCodec = - ((CompressionCodec) ReflectionUtil.createCompressionCodec(mConfig.getCompressionCodec())); + mCodec = CompressionUtil.createCompressionCodec(mConfig.getCompressionCodec()); mFileExtension = mCodec.getDefaultExtension(); - } else { + } + if (mConfig.getFileExtension() != null && !mConfig.getFileExtension().isEmpty()) { + mFileExtension = mConfig.getFileExtension(); + } else if (mFileExtension == null){ mFileExtension = ""; } + + mLocalPrefix = mConfig.getLocalPath() + '/' + IdUtil.getLocalMessageDir(); + mGeneration = mConfig.getGeneration(); } - private void adjustOffset(ParsedMessage message) throws IOException { + public void adjustOffset(Message message) throws IOException { TopicPartition topicPartition = new TopicPartition(message.getTopic(), message.getKafkaPartition()); long lastSeenOffset = mOffsetTracker.getLastSeenOffset(topicPartition); if (message.getOffset() != lastSeenOffset + 1) { + StatsUtil.incr("secor.consumer_rebalance_count." + topicPartition.getTopic()); // 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()); + LOG.debug("offset of message {} does not follow sequentially the last seen offset {}. " + + "Deleting files in topic {} partition {}", + message, lastSeenOffset, topicPartition.getTopic(), topicPartition.getPartition()); + mFileRegistry.deleteTopicPartition(topicPartition); } mOffsetTracker.setLastSeenOffset(topicPartition, message.getOffset()); } - public void write(ParsedMessage message) throws IOException { - adjustOffset(message); + public void write(ParsedMessage message) throws Exception { 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, mFileExtension); - LongWritable key = new LongWritable(message.getOffset()); - BytesWritable value = new BytesWritable(message.getPayload()); - SequenceFile.Writer writer; - writer = mFileRegistry.getOrCreateWriter(path, mCodec); - writer.append(key, value); - LOG.debug("appended message " + message + " to file " + path.getLogFilePath() + - ". File length " + writer.getLength()); + LogFilePath path = new LogFilePath(mLocalPrefix, mGeneration, offset, message, + mFileExtension); + FileWriter writer = mFileRegistry.getOrCreateWriter(path, mCodec); + writer.write(new KeyValue(message.getOffset(), message.getKafkaKey(), message.getPayload(), message.getTimestamp())); + LOG.debug("appended message {} to file {}. File length {}", + message, path, writer.getLength()); } } diff --git a/src/main/scripts/docker-entrypoint.sh b/src/main/scripts/docker-entrypoint.sh new file mode 100644 index 000000000..5052cfb81 --- /dev/null +++ b/src/main/scripts/docker-entrypoint.sh @@ -0,0 +1,100 @@ +#!/bin/bash +set -e + + +SECOR_CONFIG='' + +if [ -z "$ZOOKEEPER_QUORUM" ]; then + echo "ZOOKEEPER_QUORUM variable not set, launch with -e ZOOKEEPER_QUORUM=zookeeper:2181" + exit 1 +else + SECOR_CONFIG="$SECOR_CONFIG -Dzookeeper.quorum=$ZOOKEEPER_QUORUM" + echo "zookeeper.quorum=$ZOOKEEPER_QUORUM" +fi + +if [ -z "$ZOOKEEPER_PATH" ]; then + echo "ZOOKEEPER_PATH variable not set, launch with -e ZOOKEEPER_PATH=/" + exit 1 +else + SECOR_CONFIG="$SECOR_CONFIG -Dkafka.zookeeper.path=$ZOOKEEPER_PATH" + echo "kafka.zookeeper.path=$ZOOKEEPER_PATH" +fi + +if [[ ! -z "$KAFKA_SEED_BROKER_HOST" ]]; then + SECOR_CONFIG="$SECOR_CONFIG -Dkafka.seed.broker.host=$KAFKA_SEED_BROKER_HOST" + echo "kafka.seed.broker.host=$KAFKA_SEED_BROKER_HOST" +fi +if [[ ! -z "$KAFKA_SEED_BROKER_PORT" ]]; then + SECOR_CONFIG="$SECOR_CONFIG -Dkafka.seed.broker.port=$KAFKA_SEED_BROKER_PORT" + echo "kafka.seed.broker.port=$KAFKA_SEED_BROKER_PORT" +fi + +if [[ ! -z "$SECOR_GROUP" ]]; then + SECOR_CONFIG="$SECOR_CONFIG -Dsecor.kafka.group=$SECOR_GROUP" + echo "secor.kafka.group=$SECOR_GROUP" +fi + + +if [[ ! -z "$AWS_REGION" ]]; then + SECOR_CONFIG="$SECOR_CONFIG -Daws.region=$AWS_REGION" + echo "aws.region=$AWS_REGION" +fi +if [[ ! -z "$AWS_ENDPOINT" ]]; then + SECOR_CONFIG="$SECOR_CONFIG -Daws.endpoint=$AWS_ENDPOINT" + echo "aws.endpoint=$AWS_ENDPOINT" +fi +if [[ ! -z "$AWS_PATH_STYLE_ACCESS" ]]; then + SECOR_CONFIG="$SECOR_CONFIG -Daws.client.pathstyleaccess=$AWS_PATH_STYLE_ACCESS" + echo "aws.client.pathstyleaccess=$AWS_PATH_STYLE_ACCESS" +fi +if [[ ! -z "$AWS_ACCESS_KEY" ]]; then + SECOR_CONFIG="$SECOR_CONFIG -Daws.access.key=$AWS_ACCESS_KEY" +fi +if [[ ! -z "$AWS_SECRET_KEY" ]]; then + SECOR_CONFIG="$SECOR_CONFIG -Daws.secret.key=$AWS_SECRET_KEY" +fi +if [[ ! -z "$SECOR_S3_BUCKET" ]]; then + SECOR_CONFIG="$SECOR_CONFIG -Dsecor.s3.bucket=$SECOR_S3_BUCKET" + echo "secor.s3.bucket=$SECOR_S3_BUCKET" +fi +if [[ ! -z "$S3_PATH" ]]; then + SECOR_CONFIG="$SECOR_CONFIG -Dsecor.s3.path=$S3_PATH" + echo "secor.s3.path=$S3_PATH" +fi + + + +if [[ ! -z "$SECOR_MAX_FILE_BYTES" ]]; then + SECOR_CONFIG="$SECOR_CONFIG -Dsecor.max.file.size.bytes=$SECOR_MAX_FILE_BYTES" + echo "secor.max.file.size.bytes=$SECOR_MAX_FILE_BYTES" +fi +if [[ ! -z "$SECOR_MAX_FILE_SECONDS" ]]; then + SECOR_CONFIG="$SECOR_CONFIG -Dsecor.max.file.age.seconds=$SECOR_MAX_FILE_SECONDS" + echo "secor.max.file.age.seconds=$SECOR_MAX_FILE_SECONDS" +fi + + +if [[ ! -z "$SECOR_KAFKA_TOPIC_FILTER" ]]; then + SECOR_CONFIG="$SECOR_CONFIG -Dsecor.kafka.topic_filter=$SECOR_KAFKA_TOPIC_FILTER" + echo "secor.kafka.topic_filter=$SECOR_KAFKA_TOPIC_FILTER" +fi +if [[ ! -z "$SECOR_WRITER_FACTORY" ]]; then + SECOR_CONFIG="$SECOR_CONFIG -Dsecor.file.reader.writer.factory=$SECOR_WRITER_FACTORY" + echo "secor.file.reader.writer.factory=$SECOR_WRITER_FACTORY" +fi +if [[ ! -z "$SECOR_MESSAGE_PARSER" ]]; then + SECOR_CONFIG="$SECOR_CONFIG -Dsecor.message.parser.class=$SECOR_MESSAGE_PARSER" + echo "secor.message.parser.class=$SECOR_MESSAGE_PARSER" +fi +SECOR_CONFIG="$SECOR_CONFIG $SECOR_EXTRA_OPTS" + + +cd /opt/secor + + +DEFAULT_CLASSPATH="*:lib/*" +CLASSPATH=${CLASSPATH:-$DEFAULT_CLASSPATH} + +java -Xmx${JVM_MEMORY:-512m} $JAVA_OPTS -ea -Dsecor_group=${SECOR_GROUP:-partition} -Dlog4j.configuration=file:./${LOG4J_CONFIGURATION:-log4j.docker.properties} \ + -Dconfig=secor.prod.partition.properties $SECOR_CONFIG \ + -cp $CLASSPATH com.pinterest.secor.main.ConsumerMain diff --git a/src/main/scripts/run_common.sh b/src/main/scripts/run_common.sh new file mode 100755 index 000000000..73b21a697 --- /dev/null +++ b/src/main/scripts/run_common.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash + +# Which java to use +if [ -z "${JAVA_HOME}" ]; then + # try to use Java7 by default + JAVA_HOME=/usr/lib/jvm/java-7-oracle + if [ -e $JAVA_HOME ]; then + JAVA=${JAVA_HOME}/bin/java + else + JAVA="java" + fi +else + JAVA="${JAVA_HOME}/bin/java" +fi + +DEFAULT_CLASSPATH="*:lib/*" +CLASSPATH=${CLASSPATH:-$DEFAULT_CLASSPATH} + diff --git a/src/main/scripts/run_consumer.sh b/src/main/scripts/run_consumer.sh index 834cc71f5..6886a5af0 100755 --- a/src/main/scripts/run_consumer.sh +++ b/src/main/scripts/run_consumer.sh @@ -1,4 +1,4 @@ -#!/bin/sh +#!/usr/bin/env bash # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -19,12 +19,15 @@ mkdir -p /mnt/secor_data/logs +CURR_DIR=`dirname $0` +source ${CURR_DIR}/run_common.sh + echo "starting backup group" -nohup java -ea -Dsecor_group=backup -Dlog4j.configuration=log4j.prod.properties \ +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 \ +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 index 86764f5ad..5788ca7c7 100755 --- a/src/main/scripts/run_kafka_class.sh +++ b/src/main/scripts/run_kafka_class.sh @@ -24,22 +24,6 @@ if [ $# -lt 1 ]; then 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 " @@ -49,12 +33,8 @@ 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 +CURR_DIR=`dirname $0` +source ${CURR_DIR}/run_common.sh # Memory options KAFKA_HEAP_OPTS="-Xmx256M" diff --git a/src/main/scripts/run_tests.sh b/src/main/scripts/run_tests.sh index 9af93ac3f..e41162e14 100755 --- a/src/main/scripts/run_tests.sh +++ b/src/main/scripts/run_tests.sh @@ -25,7 +25,8 @@ # mvn package # mkdir /tmp/test # cd /tmp/test -# tar -zxvf ~/git/optimus/secor/target/secor-0.1-SNAPSHOT-bin.tar.gz +# tar -zxvf ~/git/optimus/secor/target/secor-0.2-SNAPSHOT-bin.tar.gz +# # # copy Hadoop native libs to lib/, or change HADOOP_NATIVE_LIB_PATH to point to them # ./scripts/run_tests.sh # @@ -34,118 +35,223 @@ # 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 +BUCKET=${SECOR_BUCKET:-test-bucket} +S3_LOGS_DIR=s3://${BUCKET}/secor_dev +SWIFT_CONTAINER=logsContainer +# Should match the secor.swift.containers.for.each.topic value +CONTAINER_PER_TOPIC=false +MESSAGES=100 +MESSAGE_TYPE=binary # For the compression tests to work, set this to the path of the Hadoop native libs. HADOOP_NATIVE_LIB_PATH=lib +# by default additional opts is empty +ADDITIONAL_OPTS= + +# various reader writer options to be used for testing +# note associate array needs bash v4 support +# +declare -A READER_WRITERS +READER_WRITERS[json]=com.pinterest.secor.io.impl.DelimitedTextFileReaderWriterFactory +READER_WRITERS[binary]=com.pinterest.secor.io.impl.SequenceFileReaderWriterFactory + +# Hadoop supports multiple implementations of the s3 filesystem +S3_FILESYSTEMS=${S3_FILESYSTEMS:-s3n} + +# The minimum wait time is 10 seconds plus delta. Secor is configured to upload files older than +# 10 seconds and we need to make sure that everything ends up on s3 before starting verification. +WAIT_TIME=${SECOR_WAIT_TIME:-40} +BASE_DIR=$(dirname $0) +CONF_DIR=${BASE_DIR}/.. + +cloudService="s3" +if [ "$#" != "0" ]; then + cloudService=${1} +fi -# 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) +source ${BASE_DIR}/run_common.sh run_command() { echo "running $@" eval "$@" } +check_for_native_libs() { + files=($(find "${HADOOP_NATIVE_LIB_PATH}" -maxdepth 1 -name "*.so" 2> /dev/null)) + if [ ${#files[@]} -eq 0 ]; then + echo "Couldn't find Hadoop native libraries, skipping compressed binary tests" + SKIP_COMPRESSED_BINARY="true" + fi +} + recreate_dirs() { run_command "rm -r -f ${PARENT_DIR}" - run_command "s3cmd del --recursive ${S3_LOGS_DIR}" + if [ ${cloudService} = "swift" ]; then + if ${CONTAINER_PER_TOPIC}; then + run_command "swift delete test" + else + run_command "swift delete ${SWIFT_CONTAINER}" + sleep 3 + run_command "swift post ${SWIFT_CONTAINER}" + fi + else + if [ -n "${SECOR_LOCAL_S3}" ]; then + run_command "s3cmd -c ${CONF_DIR}/test.s3cfg ls -r ${S3_LOGS_DIR} | awk '{ print \$4 }' | xargs -L 1 s3cmd -c ${CONF_DIR}/test.s3cfg del" + run_command "s3cmd -c ${CONF_DIR}/test.s3cfg ls -r ${S3_LOGS_DIR}" + else + run_command "s3cmd del --recursive ${S3_LOGS_DIR}" + run_command "s3cmd ls -r ${S3_LOGS_DIR}" + fi + fi # create logs directory if [ ! -d ${LOGS_DIR} ]; then run_command "mkdir -p ${LOGS_DIR}" fi } +start_s3() { + if [ -n "${SECOR_LOCAL_S3}" ]; then + if command -v fakes3 > /dev/null 2>&1; then + run_command "fakes3 --root=/tmp/fakes3 --port=5000 --hostname=localhost > /tmp/fakes3.log 2>&1 &" + sleep 10 + run_command "s3cmd -c ${CONF_DIR}/test.s3cfg mb s3://${BUCKET}" + else + echo "Couldn't find FakeS3 binary, please install it using `gem install fakes3`" + fi + fi +} + +stop_s3() { + if [ -n "${SECOR_LOCAL_S3}" ]; then + run_command "pkill -f 'fakes3' || true" + run_command "rm -r -f /tmp/fakes3" + fi +} + start_zookeeper() { - run_command "${base_dir}/run_kafka_class.sh \ - org.apache.zookeeper.server.quorum.QuorumPeerMain zookeeper.test.properties > \ + run_command "${BASE_DIR}/run_kafka_class.sh \ + org.apache.zookeeper.server.quorum.QuorumPeerMain ${CONF_DIR}/zookeeper.test.properties > \ ${LOGS_DIR}/zookeeper.log 2>&1 &" } stop_zookeeper() { - run_command "pkill -f 'org.apache.zookeeper.server.quorum.QuorumPeerMain' | true" + run_command "pkill -f 'org.apache.zookeeper.server.quorum.QuorumPeerMain' || true" } start_kafka_server () { - run_command "${base_dir}/run_kafka_class.sh kafka.Kafka kafka.test.properties > \ + run_command "${BASE_DIR}/run_kafka_class.sh kafka.Kafka ${CONF_DIR}/kafka.test.properties > \ ${LOGS_DIR}/kafka_server.log 2>&1 &" } stop_kafka_server() { - run_command "pkill -f 'kafka.Kafka' | true" + run_command "pkill -f 'kafka.Kafka' || true" } start_secor() { - run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ - -Dconfig=secor.dev.backup.properties -cp secor-0.1-SNAPSHOT.jar:lib/* \ + run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ + -Dconfig=secor.test.backup.properties ${ADDITIONAL_OPTS} -cp $CLASSPATH \ 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 &" + if [ "${MESSAGE_TYPE}" = "binary" ]; then + run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ + -Dconfig=secor.test.partition.properties ${ADDITIONAL_OPTS} -cp $CLASSPATH \ + com.pinterest.secor.main.ConsumerMain > ${LOGS_DIR}/secor_partition.log 2>&1 &" + fi } -start_secor_compressed() { - run_command "java -server -ea -Dlog4j.configuration=log4j.dev.properties \ - -Dconfig=secor.dev.backup.properties -Djava.library.path=$HADOOP_NATIVE_LIB_PATH \ - -Dsecor.compression.codec=org.apache.hadoop.io.compress.GzipCodec \ - -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 -Djava.library.path=$HADOOP_NATIVE_LIB_PATH \ - -Dsecor.compression.codec=org.apache.hadoop.io.compress.GzipCodec \ - -cp secor-0.1-SNAPSHOT.jar:lib/* \ - com.pinterest.secor.main.ConsumerMain > ${LOGS_DIR}/secor_partition.log 2>&1 &" +stop_secor() { + run_command "pkill -f 'com.pinterest.secor.main.ConsumerMain' || true" } -stop_secor() { - run_command "pkill -f 'com.pinterest.secor.main.ConsumerMain' | true" +run_finalizer() { + run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ + -Dconfig=secor.test.partition.properties ${ADDITIONAL_OPTS} -cp $CLASSPATH \ + com.pinterest.secor.main.PartitionFinalizerMain > ${LOGS_DIR}/finalizer.log 2>&1 " + + EXIT_CODE=$? + if [ ${EXIT_CODE} -ne 0 ]; then + echo -e "\e[1;41;97mFinalizer FAILED\e[0m" + echo "See log ${LOGS_DIR}/finalizer.log for more details" + exit ${EXIT_CODE} + fi } create_topic() { - run_command "${base_dir}/run_kafka_class.sh kafka.admin.TopicCommand --create --zookeeper \ + run_command "${BASE_DIR}/run_kafka_class.sh kafka.admin.TopicCommand --create --zookeeper \ localhost:2181 --replication-factor 1 --partitions 2 --topic test > \ ${LOGS_DIR}/create_topic.log 2>&1" } +# post messages +# $1 number of messages +# $2 timeshift in seconds 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 > \ + run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ + -Dconfig=secor.test.backup.properties -cp ${CLASSPATH} \ + com.pinterest.secor.main.TestLogMessageProducerMain -t test -m $1 -p 1 -type ${MESSAGE_TYPE} -timeshift $2 > \ ${LOGS_DIR}/test_log_message_producer.log 2>&1" } +# verify the messages +# $1: number of messages +# $2: number of _SUCCESS files 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" + echo "Verifying $1 $2" + + RUNMODE_0="backup" + if [ "${MESSAGE_TYPE}" = "binary" ]; then + RUNMODE_1="partition" + else + RUNMODE_1="backup" + fi + for RUNMODE in ${RUNMODE_0} ${RUNMODE_1}; do + run_command "${JAVA} -server -ea -Dlog4j.configuration=log4j.dev.properties \ + -Dconfig=secor.test.${RUNMODE}.properties ${ADDITIONAL_OPTS} -cp ${CLASSPATH} \ + com.pinterest.secor.main.LogFileVerifierMain -t test -m $1 -q > \ + ${LOGS_DIR}/log_verifier_${RUNMODE}.log 2>&1" + VERIFICATION_EXIT_CODE=$? + if [ ${VERIFICATION_EXIT_CODE} -ne 0 ]; then + echo -e "\e[1;41;97mVerification FAILED\e[0m" + echo "See log ${LOGS_DIR}/log_verifier_${RUNMODE}.log for more details" + tail -n 50 ${LOGS_DIR}/log_verifier_${RUNMODE}.log + echo "See log ${LOGS_DIR}/secor_${RUNMODE}.log for more details" + tail -n 50 ${LOGS_DIR}/secor_${RUNMODE}.log + echo "See log ${LOGS_DIR}/test_log_message_producer.log for more details" + tail -n 50 ${LOGS_DIR}/test_log_message_producer.log + exit ${VERIFICATION_EXIT_CODE} + fi + + # Verify SUCCESS file + if [ ${cloudService} = "swift" ]; then + run_command "swift list ${SWIFT_CONTAINER} | grep _SUCCESS | wc -l > /tmp/secor_tests_output.txt" + else + if [ -n "${SECOR_LOCAL_S3}" ]; then + run_command "s3cmd ls -c ${CONF_DIR}/test.s3cfg -r ${S3_LOGS_DIR} | grep _SUCCESS | wc -l > /tmp/secor_tests_output.txt" + else + run_command "s3cmd ls -r ${S3_LOGS_DIR} | grep _SUCCESS | wc -l > /tmp/secor_tests_output.txt" + fi + fi + count=$( ${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" + cat < ${LOGS_DIR}/run_zookeeper_command.log 2>&1" +create /consumers '' +create /consumers/${group} '' +create /consumers/${group}/offsets '' +create /consumers/${group}/offsets/test '' +create /consumers/${group}/offsets/test/${partition} $1 +quit +EOF done done } @@ -174,76 +280,180 @@ initialize() { # Post some messages and verify that they are correctly processed. post_and_verify_test() { + echo "********************************************************" echo "running post_and_verify_test" initialize start_secor sleep 3 - post_messages ${MESSAGES} + post_messages ${MESSAGES} 0 + echo "Waiting ${WAIT_TIME} sec for Secor to upload logs to s3" + sleep ${WAIT_TIME} + verify ${MESSAGES} 0 + + stop_all + echo -e "\e[1;42;97mpost_and_verify_test succeeded\e[0m" +} + +# Post some messages and run the finalizer, count # of messages and success file +# $1: hr or dt, decides whether it's hourly or daily folder finalization +post_and_finalizer_verify_test() { + echo "********************************************************" + date=$(date -u +'%Y-%m-%d %H:%M:%S') + read Y M D h m s <<< ${date//[-: ]/ } + if [ $m -ge 55 ]; then + # we have to know the number of hr/dt folders to be created + echo "It's too close to the hour mark: $m, skip the test" + return + fi + if [ $h -le 1 ]; then + # This will make the timeshift pass the day boundary + echo "It's too close to the day mark: $h, skip the test" + return + fi + + HOUR_TIMESHIFT=$((3600+3600)) + DAY_TIMESHIFT=$((86400+3600)) + + OLD_ADDITIONAL_OPTS=${ADDITIONAL_OPTS} + + if [ $1 = "hr" ]; then + ADDITIONAL_OPTS="${ADDITIONAL_OPTS} -Dpartitioner.granularity.hour=true -Dsecor.finalizer.lookback.periods=30" + # should be 2 success files for hr folder, 1 for dt folder + FILES=3 + else + # should be 1 success files for dt folder + FILES=1 + fi + echo "Expected success file: $FILES" + + echo "running post_and_finalizer_verify_test $1" + initialize + + start_secor + sleep 3 + + # post some messages for yesterday + post_messages ${MESSAGES} ${DAY_TIMESHIFT} + # post some messages for last hour + post_messages ${MESSAGES} ${HOUR_TIMESHIFT} + # post some current messages + post_messages ${MESSAGES} 0 + echo "Waiting ${WAIT_TIME} sec for Secor to upload logs to s3" sleep ${WAIT_TIME} - verify ${MESSAGES} + + echo "start finalizer" + run_finalizer + + verify $((${MESSAGES}*3)) ${FILES} stop_all - echo "post_and_verify_test succeeded" + ADDITIONAL_OPTS=${OLD_ADDITIONAL_OPTS} + + echo -e "\e[1;42;97mpost_and_finalizer_verify_test succeeded\e[0m" } # Adjust offsets so that Secor consumes only half of the messages. start_from_non_zero_offset_test() { + echo "********************************************************" echo "running start_from_non_zero_offset_test" initialize set_offsets_in_zookeeper $((${MESSAGES}/4)) - post_messages ${MESSAGES} + post_messages ${MESSAGES} 0 start_secor echo "Waiting ${WAIT_TIME} sec for Secor to upload logs to s3" sleep ${WAIT_TIME} - verify $((${MESSAGES}/2)) + verify $((${MESSAGES}/2)) 0 stop_all - echo "start_from_non_zero_offset_test succeeded" + echo -e "\e[1;42;97mstart_from_non_zero_offset_test succeeded\e[0m" } # 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. +# re-balancing event and potential topic reassignment triggering the need to trim local log files. move_offset_back_test() { + echo "********************************************************" echo "running move_offset_back_test" initialize + OLD_ADDITIONAL_OPTS=${ADDITIONAL_OPTS} + ADDITIONAL_OPTS="${ADDITIONAL_OPTS} -Dsecor.max.file.age.seconds=30" + start_secor sleep 3 - post_messages $((${MESSAGES}/10)) + post_messages $((${MESSAGES}/10)) 0 set_offsets_in_zookeeper 2 - post_messages $((${MESSAGES}*9/10)) + post_messages $((${MESSAGES}*9/10)) 0 - 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)) + # file.age increased to 30 from 10, so multiply wait time by 3. + echo "Waiting $((${WAIT_TIME}*3)) sec for Secor to upload logs to s3" + sleep $((${WAIT_TIME}*3)) + # 4 because we skipped 2 messages per topic partition and there are 2 partitions per topic. + verify $((${MESSAGES}-4)) 0 stop_all - echo "move_offset_back_test succeeded" + ADDITIONAL_OPTS=${OLD_ADDITIONAL_OPTS} + + echo -e "\e[1;42;97mmove_offset_back_test succeeded\e[0m" } # Post some messages and verify that they are correctly processed and compressed. post_and_verify_compressed_test() { + echo "********************************************************" echo "running post_and_verify_compressed_test" initialize - start_secor_compressed + OLD_ADDITIONAL_OPTS=${ADDITIONAL_OPTS} + + # add compression options + ADDITIONAL_OPTS="${ADDITIONAL_OPTS} -Dsecor.compression.codec=org.apache.hadoop.io.compress.GzipCodec \ + -Djava.library.path=$HADOOP_NATIVE_LIB_PATH" + start_secor sleep 3 - post_messages ${MESSAGES} + post_messages ${MESSAGES} 0 echo "Waiting ${WAIT_TIME} sec for Secor to upload logs to s3" sleep ${WAIT_TIME} - verify ${MESSAGES} + verify ${MESSAGES} 0 stop_all - echo "post_and_verify_compressed_test succeeded" -} + ADDITIONAL_OPTS=${OLD_ADDITIONAL_OPTS} + echo -e "\e[1;42;97mpost_and_verify_compressed_test succeeded\e[0m" +} +check_for_native_libs +if [ ${cloudService} = "s3" ]; then + stop_s3 + start_s3 +fi + +for fkey in ${S3_FILESYSTEMS}; do + FILESYSTEM_TYPE=${fkey} + for key in ${!READER_WRITERS[@]}; do + MESSAGE_TYPE=${key} + ADDITIONAL_OPTS="-Dsecor.s3.filesystem=${FILESYSTEM_TYPE} -Dsecor.file.reader.writer.factory=${READER_WRITERS[${key}]}" + echo "********************************************************" + echo "Running tests for Message Type: ${MESSAGE_TYPE} and ReaderWriter:${READER_WRITERS[${key}]} using filesystem: ${FILESYSTEM_TYPE}" + post_and_verify_test + if [ ${MESSAGE_TYPE} = "binary" ]; then + # Testing finalizer in partition mode + post_and_finalizer_verify_test hr + post_and_finalizer_verify_test dt + fi + start_from_non_zero_offset_test + move_offset_back_test + if [ ${MESSAGE_TYPE} = "json" ]; then + post_and_verify_compressed_test + elif [ -z ${SKIP_COMPRESSED_BINARY} ]; then + post_and_verify_compressed_test + else + echo "Skipping compressed tests for ${MESSAGE_TYPE}" + fi + done +done -post_and_verify_test -start_from_non_zero_offset_test -move_offset_back_test -post_and_verify_compressed_test +if [ ${cloudService} = "s3" ]; then + stop_s3 +fi diff --git a/src/main/scripts/run_zookeeper_command.sh b/src/main/scripts/run_zookeeper_command.sh index 8c0128e5b..873737b95 100755 --- a/src/main/scripts/run_zookeeper_command.sh +++ b/src/main/scripts/run_zookeeper_command.sh @@ -1,4 +1,4 @@ -#!/bin/sh +#!/usr/bin/env bash # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -17,9 +17,7 @@ # Author: Pawel Garbacki (pawel@pinterest.com) -if [ $# -lt 3 ]; then - echo "USAGE: $0 zookeeper_host:port cmd args" - exit 1 -fi +CURR_DIR=`dirname $0` +source ${CURR_DIR}/run_common.sh -java -ea -cp "secor-0.1-SNAPSHOT.jar:lib/*" org.apache.zookeeper.ZooKeeperMain -server $@ +${JAVA} -ea -cp "$CLASSPATH" org.apache.zookeeper.ZooKeeperMain -server $@ diff --git a/src/test/config/core-site.xml b/src/test/config/core-site.xml new file mode 100644 index 000000000..d42d3cfe1 --- /dev/null +++ b/src/test/config/core-site.xml @@ -0,0 +1,7 @@ + + + fs.s3a.endpoint + For testing override the endpoint to fakes3 + http://localhost:5000 + + diff --git a/src/test/config/jets3t.properties b/src/test/config/jets3t.properties new file mode 100644 index 000000000..18023ea5f --- /dev/null +++ b/src/test/config/jets3t.properties @@ -0,0 +1,4 @@ +s3service.https-only=false +s3service.s3-endpoint-http-port=5000 +s3service.s3-endpoint=localhost +s3service.disable-dns-buckets=true diff --git a/src/test/config/secor.kafka.migration.test.properties b/src/test/config/secor.kafka.migration.test.properties new file mode 100644 index 000000000..056ab517a --- /dev/null +++ b/src/test/config/secor.kafka.migration.test.properties @@ -0,0 +1,6 @@ +# Store offset in zookeeper and kafka consumer topic +kafka.dual.commit.enabled=false + +# Storage offset. +# Possible value "zookeeper" to read offset from zookeeper or "kafka" to read offset from kafka consumer topic +kafka.offsets.storage=kafka diff --git a/src/test/config/secor.test.monitoring.properties b/src/test/config/secor.test.monitoring.properties new file mode 100644 index 000000000..df3cb80aa --- /dev/null +++ b/src/test/config/secor.test.monitoring.properties @@ -0,0 +1 @@ +secor.monitoring.metrics.collector.class=com.pinterest.secor.monitoring.OstrichMetricCollector diff --git a/src/test/config/secor.test.protobuf.properties b/src/test/config/secor.test.protobuf.properties new file mode 100644 index 000000000..75e6dd742 --- /dev/null +++ b/src/test/config/secor.test.protobuf.properties @@ -0,0 +1,2 @@ +secor.protobuf.message.class.mytopic1=com.pinterest.secor.protobuf.Messages$UnitTestMessage1 +secor.protobuf.message.class.mytopic2=com.pinterest.secor.protobuf.Messages$UnitTestMessage2 \ No newline at end of file diff --git a/src/test/config/test.s3cfg b/src/test/config/test.s3cfg new file mode 100644 index 000000000..da36fbfb3 --- /dev/null +++ b/src/test/config/test.s3cfg @@ -0,0 +1,12 @@ +[default] +access_key = TESTACCESSKEY +bucket_location = US +default_mime_type = binary/octet-stream +encoding = UTF-8 +get_continue = False +guess_mime_type = True +host_base = localhost:5000 +host_bucket = %(bucket)s.localhost:5000 +secret_key = TESTSECRETKEY +use_https = False +verbosity = WARNING diff --git a/src/test/java/com/pinterest/secor/common/FileRegistryTest.java b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java index 897f0605a..a40844cff 100644 --- a/src/test/java/com/pinterest/secor/common/FileRegistryTest.java +++ b/src/test/java/com/pinterest/secor/common/FileRegistryTest.java @@ -16,14 +16,14 @@ */ package com.pinterest.secor.common; +import com.pinterest.secor.io.FileWriter; import com.pinterest.secor.util.FileUtil; +import com.pinterest.secor.util.ReflectionUtil; + 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.apache.commons.configuration.PropertiesConfiguration; +import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.GzipCodec; import org.junit.runner.RunWith; import org.mockito.Mockito; @@ -31,7 +31,6 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; -import java.io.IOException; import java.util.Collection; /** @@ -40,17 +39,14 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ @RunWith(PowerMockRunner.class) -@PrepareForTest({FileRegistry.class, FileSystem.class, FileUtil.class, SequenceFile.class}) +@PrepareForTest({ FileRegistry.class, FileUtil.class, ReflectionUtil.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 PATH_GZ = - "/some_parent_dir/some_topic/some_partition/some_other_partition/" + - "10_0_00000000000000000100.gz"; - private static final String CRC_PATH = - "/some_parent_dir/some_topic/some_partition/some_other_partition/" + - ".10_0_00000000000000000100.crc"; + private static final String PATH = "/some_parent_dir/some_topic/some_partition/some_other_partition/" + + "10_0_00000000000000000100"; + private static final String PATH_GZ = "/some_parent_dir/some_topic/some_partition/some_other_partition/" + + "10_0_00000000000000000100.gz"; + private static final String CRC_PATH = "/some_parent_dir/some_topic/some_partition/some_other_partition/" + + ".10_0_00000000000000000100.crc"; private LogFilePath mLogFilePath; private LogFilePath mLogFilePathGz; private TopicPartition mTopicPartition; @@ -58,88 +54,101 @@ public class FileRegistryTest extends TestCase { public void setUp() throws Exception { super.setUp(); + PropertiesConfiguration properties = new PropertiesConfiguration(); + properties.addProperty("secor.file.reader.writer.factory", + "com.pinterest.secor.io.impl.SequenceFileReaderWriterFactory"); + properties.addProperty("secor.file.age.youngest", true); + SecorConfig secorConfig = new SecorConfig(properties); + mRegistry = new FileRegistry(secorConfig); mLogFilePath = new LogFilePath("/some_parent_dir", PATH); mTopicPartition = new TopicPartition("some_topic", 0); - mRegistry = new FileRegistry(); mLogFilePathGz = new LogFilePath("/some_parent_dir", PATH_GZ); } - private void createWriter() throws IOException { + private FileWriter createWriter() throws Exception { 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); + PowerMockito.mockStatic(ReflectionUtil.class); + FileWriter writer = Mockito.mock(FileWriter.class); + Mockito.when( + ReflectionUtil.createFileWriter( + Mockito.any(String.class), + Mockito.any(LogFilePath.class), + Mockito.any(CompressionCodec.class), + Mockito.any(SecorConfig.class) + )) + .thenReturn(writer); Mockito.when(writer.getLength()).thenReturn(123L); - SequenceFile.Writer createdWriter = mRegistry.getOrCreateWriter(mLogFilePath, null); + FileWriter createdWriter = mRegistry.getOrCreateWriter( + mLogFilePath, null); assertTrue(createdWriter == writer); + + return writer; } public void testGetOrCreateWriter() throws Exception { createWriter(); - // Call the method again. This time it should return an existing writer. + // Call the method again. This time it should return an existing writer. mRegistry.getOrCreateWriter(mLogFilePath, null); // Verify that the method has been called exactly once (the default). PowerMockito.verifyStatic(); - FileSystem.get(Mockito.any(Configuration.class)); + ReflectionUtil.createFileWriter(Mockito.any(String.class), + Mockito.any(LogFilePath.class), + Mockito.any(CompressionCodec.class), + Mockito.any(SecorConfig.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(); + Collection topicPartitions = mRegistry + .getTopicPartitions(); assertEquals(1, topicPartitions.size()); assertTrue(topicPartitions.contains(topicPartition)); - Collection logFilePaths = mRegistry.getPaths(topicPartition); + Collection logFilePaths = mRegistry + .getPaths(topicPartition); assertEquals(1, logFilePaths.size()); assertTrue(logFilePaths.contains(mLogFilePath)); } - private void createCompressedWriter() throws IOException { + public void testGetWriterShowBeNullForNewFilePaths() throws Exception { + assertNull(mRegistry.getWriter(mLogFilePath)); + } + + public void testGetWriterShowBeNotNull() throws Exception { + FileWriter createdWriter = createWriter(); + + FileWriter writer = mRegistry.getWriter(mLogFilePath); + assertNotNull(writer); + assertEquals(createdWriter, writer); + } + + private void createCompressedWriter() throws Exception { 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_GZ); - 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), - Mockito.eq(SequenceFile.CompressionType.BLOCK), - Mockito.any(GzipCodec.class))).thenReturn( - writer); + PowerMockito.mockStatic(ReflectionUtil.class); + FileWriter writer = Mockito.mock(FileWriter.class); + Mockito.when( + ReflectionUtil.createFileWriter( + Mockito.any(String.class), + Mockito.any(LogFilePath.class), + Mockito.any(CompressionCodec.class), + Mockito.any(SecorConfig.class) + )) + .thenReturn(writer); Mockito.when(writer.getLength()).thenReturn(123L); - SequenceFile.Writer createdWriter = mRegistry.getOrCreateWriter(mLogFilePathGz, new GzipCodec()); + FileWriter createdWriter = mRegistry.getOrCreateWriter( + mLogFilePathGz, new GzipCodec()); assertTrue(createdWriter == writer); } @@ -149,29 +158,26 @@ public void testGetOrCreateWriterCompressed() throws Exception { mRegistry.getOrCreateWriter(mLogFilePathGz, new GzipCodec()); // Verify that the method has been called exactly once (the default). - PowerMockito.verifyStatic(); - FileSystem.get(Mockito.any(Configuration.class)); - PowerMockito.verifyStatic(); FileUtil.delete(PATH_GZ); PowerMockito.verifyStatic(); FileUtil.delete(CRC_PATH); - Path fsPath = new Path(PATH_GZ); PowerMockito.verifyStatic(); - SequenceFile.createWriter(Mockito.any(FileSystem.class), Mockito.any(Configuration.class), - Mockito.eq(fsPath), Mockito.eq(LongWritable.class), - Mockito.eq(BytesWritable.class), - Mockito.eq(SequenceFile.CompressionType.BLOCK), - Mockito.any(GzipCodec.class) + ReflectionUtil.createFileWriter(Mockito.any(String.class), + Mockito.any(LogFilePath.class), + Mockito.any(CompressionCodec.class), + Mockito.any(SecorConfig.class) ); TopicPartition topicPartition = new TopicPartition("some_topic", 0); - Collection topicPartitions = mRegistry.getTopicPartitions(); + Collection topicPartitions = mRegistry + .getTopicPartitions(); assertEquals(1, topicPartitions.size()); assertTrue(topicPartitions.contains(topicPartition)); - Collection logFilePaths = mRegistry.getPaths(topicPartition); + Collection logFilePaths = mRegistry + .getPaths(topicPartition); assertEquals(1, logFilePaths.size()); assertTrue(logFilePaths.contains(mLogFilePath)); } @@ -214,7 +220,8 @@ public void testGetSize() throws Exception { public void testGetModificationAgeSec() throws Exception { PowerMockito.mockStatic(System.class); - PowerMockito.when(System.currentTimeMillis()).thenReturn(10000L).thenReturn(100000L); + 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 index 2d76b9cdf..1f8b1c89a 100644 --- a/src/test/java/com/pinterest/secor/common/LogFilePathTest.java +++ b/src/test/java/com/pinterest/secor/common/LogFilePathTest.java @@ -41,17 +41,19 @@ public class LogFilePathTest extends TestCase { ".10_0_00000000000000000100.crc"; private LogFilePath mLogFilePath; + private long timestamp; @Override protected void setUp() throws Exception { super.setUp(); mLogFilePath = new LogFilePath(PREFIX, TOPIC, PARTITIONS, GENERATION, KAFKA_PARTITION, LAST_COMMITTED_OFFSET, ""); + timestamp = System.currentTimeMillis(); } public void testConstructFromMessage() throws Exception { - ParsedMessage message = new ParsedMessage(TOPIC, KAFKA_PARTITION, 1000, - "some_payload".getBytes(), PARTITIONS); + ParsedMessage message = new ParsedMessage(TOPIC, KAFKA_PARTITION, 1000, null, + "some_payload".getBytes(), PARTITIONS, timestamp); LogFilePath logFilePath = new LogFilePath(PREFIX, GENERATION, LAST_COMMITTED_OFFSET, message, ""); assertEquals(PATH, logFilePath.getLogFilePath()); diff --git a/src/test/java/com/pinterest/secor/common/SecorConfigTest.java b/src/test/java/com/pinterest/secor/common/SecorConfigTest.java new file mode 100644 index 000000000..6e2e7f834 --- /dev/null +++ b/src/test/java/com/pinterest/secor/common/SecorConfigTest.java @@ -0,0 +1,62 @@ +package com.pinterest.secor.common; + +import com.pinterest.secor.protobuf.Messages.UnitTestMessage1; +import com.pinterest.secor.protobuf.Messages.UnitTestMessage2; +import org.apache.commons.configuration.ConfigurationException; +import org.apache.commons.configuration.PropertiesConfiguration; +import org.junit.Test; + +import java.net.URL; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class SecorConfigTest { + + @Test + public void config_should_read_migration_required_properties_default_values() throws ConfigurationException { + + URL configFile = Thread.currentThread().getContextClassLoader().getResource("secor.common.properties"); + PropertiesConfiguration properties = new PropertiesConfiguration(configFile); + + SecorConfig secorConfig = new SecorConfig(properties); + assertEquals("true", secorConfig.getDualCommitEnabled()); + assertEquals("zookeeper", secorConfig.getOffsetsStorage()); + } + + @Test + public void config_should_read_migration_required() throws ConfigurationException { + + URL configFile = Thread.currentThread().getContextClassLoader().getResource("secor.kafka.migration.test.properties"); + PropertiesConfiguration properties = new PropertiesConfiguration(configFile); + + SecorConfig secorConfig = new SecorConfig(properties); + assertEquals("false", secorConfig.getDualCommitEnabled()); + assertEquals("kafka", secorConfig.getOffsetsStorage()); + } + + @Test + public void testProtobufMessageClassPerTopic() throws ConfigurationException { + + URL configFile = Thread.currentThread().getContextClassLoader().getResource("secor.test.protobuf.properties"); + PropertiesConfiguration properties = new PropertiesConfiguration(configFile); + + SecorConfig secorConfig = new SecorConfig(properties); + Map messageClassPerTopic = secorConfig.getProtobufMessageClassPerTopic(); + + assertEquals(2, messageClassPerTopic.size()); + assertEquals(UnitTestMessage1.class.getName(), messageClassPerTopic.get("mytopic1")); + assertEquals(UnitTestMessage2.class.getName(), messageClassPerTopic.get("mytopic2")); + } + + @Test + public void shouldReadMetricCollectorConfiguration() throws ConfigurationException { + + URL configFile = Thread.currentThread().getContextClassLoader().getResource("secor.test.monitoring.properties"); + PropertiesConfiguration properties = new PropertiesConfiguration(configFile); + + SecorConfig secorConfig = new SecorConfig(properties); + + assertEquals("com.pinterest.secor.monitoring.OstrichMetricCollector", secorConfig.getMetricsCollectorClass()); + } +} diff --git a/src/test/java/com/pinterest/secor/common/ZookeeperConnectorTest.java b/src/test/java/com/pinterest/secor/common/ZookeeperConnectorTest.java new file mode 100644 index 000000000..e7c2653ca --- /dev/null +++ b/src/test/java/com/pinterest/secor/common/ZookeeperConnectorTest.java @@ -0,0 +1,30 @@ +package com.pinterest.secor.common; + +import org.apache.commons.configuration.PropertiesConfiguration; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class ZookeeperConnectorTest { + + @Before + public void setUp() throws Exception { + } + + @Test + public void testGetCommittedOffsetGroupPath() throws Exception { + verify("/", "/consumers/secor_cg/offsets"); + verify("/chroot", "/chroot/consumers/secor_cg/offsets"); + verify("/chroot/", "/chroot/consumers/secor_cg/offsets"); + } + + protected void verify(String zookeeperPath, String expectedOffsetPath) { + ZookeeperConnector zookeeperConnector = new ZookeeperConnector(); + PropertiesConfiguration properties = new PropertiesConfiguration(); + properties.setProperty("kafka.zookeeper.path", zookeeperPath); + properties.setProperty("secor.kafka.group", "secor_cg"); + SecorConfig secorConfig = new SecorConfig(properties); + zookeeperConnector.setConfig(secorConfig); + Assert.assertEquals(expectedOffsetPath, zookeeperConnector.getCommittedOffsetGroupPath()); + } +} \ No newline at end of file diff --git a/src/test/java/com/pinterest/secor/io/FileReaderWriterFactoryTest.java b/src/test/java/com/pinterest/secor/io/FileReaderWriterFactoryTest.java new file mode 100644 index 000000000..54896d63a --- /dev/null +++ b/src/test/java/com/pinterest/secor/io/FileReaderWriterFactoryTest.java @@ -0,0 +1,248 @@ +/** + * 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.io; + +import java.io.*; +import java.net.URI; + +import org.apache.commons.configuration.PropertiesConfiguration; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +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.apache.hadoop.io.compress.CompressionInputStream; +import org.apache.hadoop.io.compress.CompressionOutputStream; +import org.apache.hadoop.io.compress.GzipCodec; +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 com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.io.impl.DelimitedTextFileReaderWriterFactory; +import com.pinterest.secor.io.impl.SequenceFileReaderWriterFactory; +import com.pinterest.secor.util.ReflectionUtil; + +import junit.framework.TestCase; + +/** + * Test the file readers and writers + * + * @author Praveen Murugesan (praveen@uber.com) + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({FileSystem.class, DelimitedTextFileReaderWriterFactory.class, + SequenceFile.class, SequenceFileReaderWriterFactory.class, GzipCodec.class, + FileInputStream.class, FileOutputStream.class}) +public class FileReaderWriterFactoryTest extends TestCase { + + private static final String DIR = "/some_parent_dir/some_topic/some_partition/some_other_partition"; + private static final String BASENAME = "10_0_00000000000000000100"; + private static final String PATH = DIR + "/" + BASENAME; + private static final String PATH_GZ = DIR + "/" + BASENAME + ".gz"; + + private LogFilePath mLogFilePath; + private LogFilePath mLogFilePathGz; + private SecorConfig mConfig; + + @Override + public void setUp() throws Exception { + super.setUp(); + mLogFilePath = new LogFilePath("/some_parent_dir", PATH); + mLogFilePathGz = new LogFilePath("/some_parent_dir", PATH_GZ); + } + + private void setupSequenceFileReaderConfig() { + PropertiesConfiguration properties = new PropertiesConfiguration(); + properties.addProperty("secor.file.reader.writer.factory", + "com.pinterest.secor.io.impl.SequenceFileReaderWriterFactory"); + mConfig = new SecorConfig(properties); + } + + private void setupDelimitedTextFileWriterConfig() { + PropertiesConfiguration properties = new PropertiesConfiguration(); + properties.addProperty("secor.file.reader.writer.factory", + "com.pinterest.secor.io.impl.DelimitedTextFileReaderWriterFactory"); + mConfig = new SecorConfig(properties); + } + + private void mockDelimitedTextFileWriter(boolean isCompressed) throws Exception { + PowerMockito.mockStatic(FileSystem.class); + FileSystem fs = Mockito.mock(FileSystem.class); + Mockito.when( + FileSystem.get(Mockito.any(URI.class), + Mockito.any(Configuration.class))).thenReturn(fs); + + Path fsPath = (!isCompressed) ? new Path(PATH) : new Path(PATH_GZ); + + GzipCodec codec = PowerMockito.mock(GzipCodec.class); + PowerMockito.whenNew(GzipCodec.class).withNoArguments() + .thenReturn(codec); + + FSDataInputStream fileInputStream = Mockito + .mock(FSDataInputStream.class); + FSDataOutputStream fileOutputStream = Mockito + .mock(FSDataOutputStream.class); + + Mockito.when(fs.open(fsPath)).thenReturn(fileInputStream); + Mockito.when(fs.create(fsPath)).thenReturn(fileOutputStream); + + CompressionInputStream inputStream = Mockito + .mock(CompressionInputStream.class); + CompressionOutputStream outputStream = Mockito + .mock(CompressionOutputStream.class); + Mockito.when(codec.createInputStream(Mockito.any(InputStream.class))) + .thenReturn(inputStream); + + Mockito.when(codec.createOutputStream(Mockito.any(OutputStream.class))) + .thenReturn(outputStream); + } + + private void mockSequenceFileWriter(boolean isCompressed) + throws Exception { + PowerMockito.mockStatic(FileSystem.class); + FileSystem fs = Mockito.mock(FileSystem.class); + Mockito.when( + FileSystem.get(Mockito.any(URI.class), + Mockito.any(Configuration.class))).thenReturn(fs); + + Path fsPath = (!isCompressed) ? new Path(PATH) : new Path(PATH_GZ); + + SequenceFile.Reader reader = PowerMockito + .mock(SequenceFile.Reader.class); + PowerMockito + .whenNew(SequenceFile.Reader.class) + .withParameterTypes(FileSystem.class, Path.class, + Configuration.class) + .withArguments(Mockito.eq(fs), Mockito.eq(fsPath), + Mockito.any(Configuration.class)).thenReturn(reader); + + Mockito.>when(reader.getKeyClass()).thenReturn( + (Class) LongWritable.class); + Mockito.>when(reader.getValueClass()).thenReturn( + (Class) BytesWritable.class); + + if (!isCompressed) { + PowerMockito.mockStatic(SequenceFile.class); + 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); + } else { + PowerMockito.mockStatic(SequenceFile.class); + 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), + Mockito.eq(SequenceFile.CompressionType.BLOCK), + Mockito.any(GzipCodec.class))).thenReturn(writer); + + Mockito.when(writer.getLength()).thenReturn(12L); + } + } + + public void testSequenceFileReader() throws Exception { + setupSequenceFileReaderConfig(); + mockSequenceFileWriter(false); + ReflectionUtil.createFileReader(mConfig.getFileReaderWriterFactory(), mLogFilePath, null, mConfig); + + // Verify that the method has been called exactly once (the default). + PowerMockito.verifyStatic(); + FileSystem.get(Mockito.any(URI.class), Mockito.any(Configuration.class)); + + mockSequenceFileWriter(true); + ReflectionUtil.createFileWriter(mConfig.getFileReaderWriterFactory(), mLogFilePathGz, new GzipCodec(), + mConfig); + + // Verify that the method has been called exactly once (the default). + PowerMockito.verifyStatic(); + FileSystem + .get(Mockito.any(URI.class), Mockito.any(Configuration.class)); + } + + public void testSequenceFileWriter() throws Exception { + setupSequenceFileReaderConfig(); + mockSequenceFileWriter(false); + + FileWriter writer = ReflectionUtil.createFileWriter(mConfig.getFileReaderWriterFactory(), + mLogFilePath, null, mConfig); + + // Verify that the method has been called exactly once (the default). + PowerMockito.verifyStatic(); + FileSystem + .get(Mockito.any(URI.class), Mockito.any(Configuration.class)); + + assert writer.getLength() == 123L; + + mockSequenceFileWriter(true); + + writer = ReflectionUtil.createFileWriter(mConfig.getFileReaderWriterFactory(), + mLogFilePathGz, new GzipCodec(), mConfig); + + // Verify that the method has been called exactly once (the default). + PowerMockito.verifyStatic(); + FileSystem + .get(Mockito.any(URI.class), Mockito.any(Configuration.class)); + + assert writer.getLength() == 12L; + } + + + public void testDelimitedTextFileWriter() throws Exception { + setupDelimitedTextFileWriterConfig(); + mockDelimitedTextFileWriter(false); + FileWriter writer = (FileWriter) ReflectionUtil + .createFileWriter(mConfig.getFileReaderWriterFactory(), + mLogFilePath, null, mConfig + ); + assert writer.getLength() == 0L; + + mockDelimitedTextFileWriter(true); + writer = (FileWriter) ReflectionUtil + .createFileWriter(mConfig.getFileReaderWriterFactory(), + mLogFilePathGz, new GzipCodec(), mConfig + ); + assert writer.getLength() == 0L; + } + + public void testDelimitedTextFileReader() throws Exception { + setupDelimitedTextFileWriterConfig(); + + mockDelimitedTextFileWriter(false); + + ReflectionUtil.createFileReader(mConfig.getFileReaderWriterFactory(), mLogFilePath, null, mConfig); + + mockDelimitedTextFileWriter(true); + ReflectionUtil.createFileReader(mConfig.getFileReaderWriterFactory(), mLogFilePathGz, new GzipCodec(), + mConfig); + } +} \ No newline at end of file diff --git a/src/test/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactoryTest.java b/src/test/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactoryTest.java new file mode 100644 index 000000000..84bad994b --- /dev/null +++ b/src/test/java/com/pinterest/secor/io/impl/MessagePackSequenceFileReaderWriterFactoryTest.java @@ -0,0 +1,78 @@ +/** + * 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.io.impl; + +import com.google.common.io.Files; +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileWriter; +import com.pinterest.secor.io.KeyValue; +import org.junit.Test; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +public class MessagePackSequenceFileReaderWriterFactoryTest { + + @Test + public void testMessagePackSequenceReadWriteRoundTrip() throws Exception { + MessagePackSequenceFileReaderWriterFactory factory = + new MessagePackSequenceFileReaderWriterFactory(); + LogFilePath tempLogFilePath = new LogFilePath(Files.createTempDir().toString(), + "test-topic", + new String[]{"part-1"}, + 0, + 1, + 0, + ".log" + ); + FileWriter fileWriter = factory.BuildFileWriter(tempLogFilePath, null); + KeyValue kv1 = (new KeyValue(23232, new byte[]{44, 55, 66, 77, 88}, new byte[]{23, 45, 40 ,10, 122})); + KeyValue kv2 = (new KeyValue(23233, new byte[]{2, 3, 4, 5})); + KeyValue kv3 = (new KeyValue(23234, new byte[]{44, 55, 66, 77, 88}, new byte[]{23, 45, 40 ,10, 122}, 1496318250l)); + KeyValue kv4 = (new KeyValue(23235, null, new byte[]{23, 45, 40 ,10, 122}, 1496318250l)); + + fileWriter.write(kv1); + fileWriter.write(kv2); + fileWriter.write(kv3); + fileWriter.write(kv4); + + fileWriter.close(); + FileReader fileReader = factory.BuildFileReader(tempLogFilePath, null); + + KeyValue kvout = fileReader.next(); + assertEquals(kv1.getOffset(), kvout.getOffset()); + assertArrayEquals(kv1.getKafkaKey(), kvout.getKafkaKey()); + assertArrayEquals(kv1.getValue(), kvout.getValue()); + kvout = fileReader.next(); + assertEquals(kv2.getOffset(), kvout.getOffset()); + assertArrayEquals(kv2.getKafkaKey(), kvout.getKafkaKey()); + assertArrayEquals(kv2.getValue(), kvout.getValue()); + kvout = fileReader.next(); + assertEquals(kv3.getOffset(), kvout.getOffset()); + assertArrayEquals(kv3.getKafkaKey(), kvout.getKafkaKey()); + assertArrayEquals(kv3.getValue(), kvout.getValue()); + assertEquals(kv3.getTimestamp(), kvout.getTimestamp()); + kvout = fileReader.next(); + assertEquals(kv4.getOffset(), kvout.getOffset()); + assertArrayEquals(new byte[0], kvout.getKafkaKey()); + assertArrayEquals(kv4.getValue(), kvout.getValue()); + assertEquals(kv4.getTimestamp(), kvout.getTimestamp()); + + } + +} \ No newline at end of file diff --git a/src/test/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactoryTest.java b/src/test/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactoryTest.java new file mode 100644 index 000000000..8aa4849f4 --- /dev/null +++ b/src/test/java/com/pinterest/secor/io/impl/ProtobufParquetFileReaderWriterFactoryTest.java @@ -0,0 +1,99 @@ +/** + * 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.io.impl; + +import static org.junit.Assert.assertArrayEquals; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.parquet.hadoop.ParquetWriter; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.modules.junit4.PowerMockRunner; + +import com.google.common.io.Files; +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileWriter; +import com.pinterest.secor.io.KeyValue; +import com.pinterest.secor.protobuf.Messages.UnitTestMessage3; +import com.pinterest.secor.util.ParquetUtil; +import com.pinterest.secor.util.ReflectionUtil; + + +import junit.framework.TestCase; + +@RunWith(PowerMockRunner.class) +public class ProtobufParquetFileReaderWriterFactoryTest extends TestCase { + + private SecorConfig config; + + @Override + public void setUp() throws Exception { + config = Mockito.mock(SecorConfig.class); + } + + @Test + public void testProtobufParquetReadWriteRoundTrip() throws Exception { + Map classPerTopic = new HashMap(); + classPerTopic.put("test-pb-topic", UnitTestMessage3.class.getName()); + Mockito.when(config.getProtobufMessageClassPerTopic()).thenReturn(classPerTopic); + Mockito.when(config.getFileReaderWriterFactory()) + .thenReturn(ProtobufParquetFileReaderWriterFactory.class.getName()); + Mockito.when(ParquetUtil.getParquetBlockSize(config)) + .thenReturn(ParquetWriter.DEFAULT_BLOCK_SIZE); + Mockito.when(ParquetUtil.getParquetPageSize(config)) + .thenReturn(ParquetWriter.DEFAULT_PAGE_SIZE); + Mockito.when(ParquetUtil.getParquetEnableDictionary(config)) + .thenReturn(ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED); + Mockito.when(ParquetUtil.getParquetValidation(config)) + .thenReturn(ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED); + + + LogFilePath tempLogFilePath = new LogFilePath(Files.createTempDir().toString(), "test-pb-topic", + new String[] { "part-1" }, 0, 1, 23232, ".log"); + + FileWriter fileWriter = ReflectionUtil.createFileWriter(config.getFileReaderWriterFactory(), tempLogFilePath, + null, config); + + UnitTestMessage3 msg1 = UnitTestMessage3.newBuilder().setData("abc").setTimestamp(1467176315L).build(); + UnitTestMessage3 msg2 = UnitTestMessage3.newBuilder().setData("XYZ").setTimestamp(1467176344L).build(); + + KeyValue kv1 = (new KeyValue(23232, msg1.toByteArray())); + KeyValue kv2 = (new KeyValue(23233, msg2.toByteArray())); + fileWriter.write(kv1); + fileWriter.write(kv2); + fileWriter.close(); + + FileReader fileReader = ReflectionUtil.createFileReader(config.getFileReaderWriterFactory(), tempLogFilePath, + null, config); + + KeyValue kvout = fileReader.next(); + assertEquals(kv1.getOffset(), kvout.getOffset()); + assertArrayEquals(kv1.getValue(), kvout.getValue()); + assertEquals(msg1.getData(), UnitTestMessage3.parseFrom(kvout.getValue()).getData()); + + kvout = fileReader.next(); + assertEquals(kv2.getOffset(), kvout.getOffset()); + assertArrayEquals(kv2.getValue(), kvout.getValue()); + assertEquals(msg2.getData(), UnitTestMessage3.parseFrom(kvout.getValue()).getData()); + } +} \ No newline at end of file diff --git a/src/test/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactoryTest.java b/src/test/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactoryTest.java new file mode 100644 index 000000000..a74b2018a --- /dev/null +++ b/src/test/java/com/pinterest/secor/io/impl/SequenceFileReaderWriterFactoryTest.java @@ -0,0 +1,64 @@ +/** + * 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.io.impl; + +import com.google.common.io.Files; +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileWriter; +import com.pinterest.secor.io.KeyValue; +import org.junit.Test; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +public class SequenceFileReaderWriterFactoryTest { + private SequenceFileReaderWriterFactory mFactory; + + public void setUp() throws Exception { + mFactory = new SequenceFileReaderWriterFactory(); + } + + @Test + public void testSequenceReadWriteRoundTrip() throws Exception { + SequenceFileReaderWriterFactory factory = new SequenceFileReaderWriterFactory(); + LogFilePath tempLogFilePath = new LogFilePath(Files.createTempDir().toString(), + "test-topic", + new String[]{"part-1"}, + 0, + 1, + 0, + ".log" + ); + FileWriter fileWriter = factory.BuildFileWriter(tempLogFilePath, null); + KeyValue kv1 = (new KeyValue(23232, new byte[]{23, 45, 40 ,10, 122})); + KeyValue kv2 = (new KeyValue(23233, new byte[]{2, 3, 4, 5})); + fileWriter.write(kv1); + fileWriter.write(kv2); + fileWriter.close(); + FileReader fileReader = factory.BuildFileReader(tempLogFilePath, null); + + KeyValue kvout = fileReader.next(); + assertEquals(kv1.getOffset(), kvout.getOffset()); + assertArrayEquals(kv1.getValue(), kvout.getValue()); + kvout = fileReader.next(); + assertEquals(kv2.getOffset(), kvout.getOffset()); + assertArrayEquals(kv2.getValue(), kvout.getValue()); + } + + +} \ No newline at end of file diff --git a/src/test/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactoryTest.java b/src/test/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactoryTest.java new file mode 100644 index 000000000..dcfe8c1f4 --- /dev/null +++ b/src/test/java/com/pinterest/secor/io/impl/ThriftParquetFileReaderWriterFactoryTest.java @@ -0,0 +1,109 @@ +/** + * 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.io.impl; + +import static org.junit.Assert.assertArrayEquals; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.parquet.hadoop.ParquetWriter; + +import org.apache.thrift.TDeserializer; +import org.apache.thrift.TSerializer; +import org.apache.thrift.protocol.TCompactProtocol; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.modules.junit4.PowerMockRunner; + +import com.google.common.io.Files; +import com.pinterest.secor.common.LogFilePath; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileWriter; +import com.pinterest.secor.io.KeyValue; +import com.pinterest.secor.thrift.UnitTestMessage; +import com.pinterest.secor.util.ParquetUtil; +import com.pinterest.secor.util.ReflectionUtil; + +import junit.framework.TestCase; + +@RunWith(PowerMockRunner.class) +public class ThriftParquetFileReaderWriterFactoryTest extends TestCase { + + private SecorConfig config; + + @Override + public void setUp() throws Exception { + config = Mockito.mock(SecorConfig.class); + } + + @Test + public void testThriftParquetReadWriteRoundTrip() throws Exception { + Map classPerTopic = new HashMap(); + classPerTopic.put("test-pb-topic", UnitTestMessage.class.getName()); + Mockito.when(config.getThriftMessageClassPerTopic()).thenReturn(classPerTopic); + Mockito.when(config.getFileReaderWriterFactory()) + .thenReturn(ThriftParquetFileReaderWriterFactory.class.getName()); + Mockito.when(config.getThriftProtocolClass()) + .thenReturn(TCompactProtocol.class.getName()); + Mockito.when(ParquetUtil.getParquetBlockSize(config)) + .thenReturn(ParquetWriter.DEFAULT_BLOCK_SIZE); + Mockito.when(ParquetUtil.getParquetPageSize(config)) + .thenReturn(ParquetWriter.DEFAULT_PAGE_SIZE); + Mockito.when(ParquetUtil.getParquetEnableDictionary(config)) + .thenReturn(ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED); + Mockito.when(ParquetUtil.getParquetValidation(config)) + .thenReturn(ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED); + + + LogFilePath tempLogFilePath = new LogFilePath(Files.createTempDir().toString(), "test-pb-topic", + new String[] { "part-1" }, 0, 1, 23232, ".log"); + + FileWriter fileWriter = ReflectionUtil.createFileWriter(config.getFileReaderWriterFactory(), tempLogFilePath, + null, config); + + UnitTestMessage msg1 = new UnitTestMessage().setRequiredField("abc").setTimestamp(1467176315L); + UnitTestMessage msg2 = new UnitTestMessage().setRequiredField("XYZ").setTimestamp(1467176344L); + + TSerializer serializer = new TSerializer(new TCompactProtocol.Factory()); + KeyValue kv1 = new KeyValue(23232, serializer.serialize(msg1)); + KeyValue kv2 = new KeyValue(23233, serializer.serialize(msg2)); + fileWriter.write(kv1); + fileWriter.write(kv2); + fileWriter.close(); + + FileReader fileReader = ReflectionUtil.createFileReader(config.getFileReaderWriterFactory(), tempLogFilePath, + null, config); + TDeserializer deserializer = new TDeserializer(new TCompactProtocol.Factory()); + + KeyValue kvout = fileReader.next(); + assertEquals(kv1.getOffset(), kvout.getOffset()); + assertArrayEquals(kv1.getValue(), kvout.getValue()); + UnitTestMessage actual = new UnitTestMessage(); + deserializer.deserialize(actual, kvout.getValue()); + assertEquals(msg1.getRequiredField(), actual.getRequiredField()); + + kvout = fileReader.next(); + assertEquals(kv2.getOffset(), kvout.getOffset()); + assertArrayEquals(kv2.getValue(), kvout.getValue()); + actual = new UnitTestMessage(); + deserializer.deserialize(actual, kvout.getValue()); + assertEquals(msg2.getRequiredField(), actual.getRequiredField()); + } +} \ No newline at end of file diff --git a/src/test/java/com/pinterest/secor/message/MessageTest.java b/src/test/java/com/pinterest/secor/message/MessageTest.java new file mode 100644 index 000000000..d4eb8c59b --- /dev/null +++ b/src/test/java/com/pinterest/secor/message/MessageTest.java @@ -0,0 +1,16 @@ +package com.pinterest.secor.message; + +import org.junit.Test; + +public class MessageTest { + + @Test + public void testNullPayload() { + Message message = new Message("testTopic", 0, 123, null, null, 0l); + System.out.println(message); + + // no assert necessary, just making sure it does not throw a + // NullPointerException + } + +} diff --git a/src/test/java/com/pinterest/secor/monitoring/OstrichMetricCollectorTest.java b/src/test/java/com/pinterest/secor/monitoring/OstrichMetricCollectorTest.java new file mode 100644 index 000000000..d8e72cefe --- /dev/null +++ b/src/test/java/com/pinterest/secor/monitoring/OstrichMetricCollectorTest.java @@ -0,0 +1,52 @@ +package com.pinterest.secor.monitoring; + +import com.twitter.ostrich.stats.Stats; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({Stats.class}) +public class OstrichMetricCollectorTest { + private OstrichMetricCollector metricCollector = new OstrichMetricCollector(); + + @Before + public void setUp() throws Exception { + PowerMockito.mockStatic(Stats.class); + } + + @Test + public void incrementByOne() throws Exception { + metricCollector.increment("expectedLabel", "ignored"); + + PowerMockito.verifyStatic(); + Stats.incr("expectedLabel"); + } + + @Test + public void increment() throws Exception { + metricCollector.increment("expectedLabel", 42, "ignored"); + + PowerMockito.verifyStatic(); + Stats.incr("expectedLabel", 42); + } + + @Test + public void metric() throws Exception { + metricCollector.metric("expectedLabel", 42.0, "ignored"); + + PowerMockito.verifyStatic(); + Stats.addMetric("expectedLabel", 42); + } + + @Test + public void gauge() throws Exception { + metricCollector.gauge("expectedLabel", 4.2, "ignored"); + + PowerMockito.verifyStatic(); + Stats.setGauge("expectedLabel", 4.2); + } +} diff --git a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java index 1766f2730..7e9f2225a 100644 --- a/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/DateMessageParserTest.java @@ -16,16 +16,15 @@ */ package com.pinterest.secor.parser; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; import junit.framework.TestCase; - import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mockito; -import org.mockito.stubbing.OngoingStubbing; import org.powermock.modules.junit4.PowerMockRunner; -import com.pinterest.secor.common.SecorConfig; -import com.pinterest.secor.message.Message; +import java.util.TimeZone; @RunWith(PowerMockRunner.class) public class DateMessageParserTest extends TestCase { @@ -35,54 +34,116 @@ public class DateMessageParserTest extends TestCase { private Message mFormat2; private Message mFormat3; private Message mInvalidDate; - private OngoingStubbing getTimestamp; + private Message mISOFormat; + private Message mNanosecondISOFormat; + private Message mNestedISOFormat; + private long timestamp; @Override public void setUp() throws Exception { mConfig = Mockito.mock(SecorConfig.class); - Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + Mockito.when(mConfig.getTimeZone()).thenReturn(TimeZone.getTimeZone("UTC")); + + timestamp = System.currentTimeMillis(); byte format1[] = "{\"timestamp\":\"2014-07-30 10:53:20\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}" - .getBytes("UTF-8"); - mFormat1 = new Message("test", 0, 0, format1); + .getBytes("UTF-8"); + mFormat1 = new Message("test", 0, 0, null, format1, timestamp); byte format2[] = "{\"timestamp\":\"2014/10/25\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}" - .getBytes("UTF-8"); - mFormat2 = new Message("test", 0, 0, format2); + .getBytes("UTF-8"); + mFormat2 = new Message("test", 0, 0, null, format2, timestamp); byte format3[] = "{\"timestamp\":\"02001.July.04 AD 12:08 PM\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}" - .getBytes("UTF-8"); - mFormat3 = new Message("test", 0, 0, format3); + .getBytes("UTF-8"); + mFormat3 = new Message("test", 0, 0, null, format3, timestamp); byte invalidDate[] = "{\"timestamp\":\"11111111\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}" - .getBytes("UTF-8"); - mInvalidDate = new Message("test", 0, 0, invalidDate); - - getTimestamp = Mockito.when(mConfig.getMessageTimestampInputPattern()); + .getBytes("UTF-8"); + mInvalidDate = new Message("test", 0, 0, null, invalidDate, timestamp); + + byte isoFormat[] = "{\"timestamp\":\"2006-01-02T15:04:05Z\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}" + .getBytes("UTF-8"); + mISOFormat = new Message("test", 0, 0, null, isoFormat, timestamp); + + byte nanosecondISOFormat[] = "{\"timestamp\":\"2006-01-02T23:59:59.999999999Z\"}" + .getBytes("UTF-8"); + mNanosecondISOFormat = new Message("test", 0, 0, null, nanosecondISOFormat, timestamp); + + byte nestedISOFormat[] = "{\"meta_data\":{\"created\":\"2016-01-11T11:50:28.647Z\"},\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}" + .getBytes("UTF-8"); + mNestedISOFormat = new Message("test", 0, 0, null, nestedISOFormat, timestamp); } @Test public void testExtractDateUsingInputPattern() throws Exception { - getTimestamp.thenReturn("yyyy-MM-dd HH:mm:ss"); + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + Mockito.when(mConfig.getString("partitioner.granularity.date.prefix", "dt=")).thenReturn("dt="); + Mockito.when(mConfig.getString("partitioner.granularity.date.format", "yyyy-MM-dd")).thenReturn("yyyy-MM-dd"); + + Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyy-MM-dd HH:mm:ss"); assertEquals("dt=2014-07-30", new DateMessageParser(mConfig).extractPartitions(mFormat1)[0]); - getTimestamp.thenReturn("yyyy/MM/d"); + Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyy/MM/d"); assertEquals("dt=2014-10-25", new DateMessageParser(mConfig).extractPartitions(mFormat2)[0]); - getTimestamp.thenReturn("yyyyy.MMMMM.dd GGG hh:mm aaa"); + Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyyy.MMMMM.dd GGG hh:mm aaa"); assertEquals("dt=2001-07-04", new DateMessageParser(mConfig).extractPartitions(mFormat3)[0]); + + Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyy-MM-dd'T'HH:mm:ss'Z'"); + assertEquals("dt=2006-01-02", new DateMessageParser(mConfig).extractPartitions(mISOFormat)[0]); + + Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyy-MM-dd'T'HH:mm:ss"); + assertEquals("dt=2006-01-02", new DateMessageParser(mConfig).extractPartitions(mISOFormat)[0]); + + Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyy-MM-dd'T'HH:mm:ss"); + assertEquals("dt=2006-01-02", new DateMessageParser(mConfig).extractPartitions(mNanosecondISOFormat)[0]); } @Test public void testExtractDateWithWrongEntries() throws Exception { + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + Mockito.when(mConfig.getString("partitioner.granularity.date.format", "yyyy-MM-dd")).thenReturn("yyyy-MM-dd"); + // invalid date - getTimestamp.thenReturn("yyyy-MM-dd HH:mm:ss"); // any pattern + Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyy-MM-dd HH:mm:ss"); // any pattern assertEquals(DateMessageParser.defaultDate, new DateMessageParser( - mConfig).extractPartitions(mInvalidDate)[0]); + mConfig).extractPartitions(mInvalidDate)[0]); // invalid pattern - getTimestamp.thenReturn("yyy-MM-dd :s"); + Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyy-MM-dd :s"); assertEquals(DateMessageParser.defaultDate, new DateMessageParser( - mConfig).extractPartitions(mFormat1)[0]); + mConfig).extractPartitions(mFormat1)[0]); + } + + @Test + public void testDatePrefix() throws Exception { + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyy-MM-dd HH:mm:ss"); + Mockito.when(mConfig.getString("partitioner.granularity.date.prefix", "dt=")).thenReturn("foo"); + Mockito.when(mConfig.getString("partitioner.granularity.date.format", "yyyy-MM-dd")).thenReturn("yyyy-MM-dd"); + + assertEquals("foo2014-07-30", new DateMessageParser(mConfig).extractPartitions(mFormat1)[0]); + } + + @Test + public void testNestedField() throws Exception { + Mockito.when(mConfig.getMessageTimestampNameSeparator()).thenReturn("."); + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("meta_data.created"); + Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyy-MM-dd'T'hh:mm:ss.SSS'Z'"); + Mockito.when(mConfig.getString("partitioner.granularity.date.prefix", "dt=")).thenReturn("dt="); + Mockito.when(mConfig.getString("partitioner.granularity.date.format", "yyyy-MM-dd")).thenReturn("yyyy-MM-dd"); + + assertEquals("dt=2016-01-11", new DateMessageParser(mConfig).extractPartitions(mNestedISOFormat)[0]); + } + + @Test + public void testCustomDateFormat() throws Exception { + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("yyyy-MM-dd HH:mm:ss"); + Mockito.when(mConfig.getString("partitioner.granularity.date.prefix", "dt=")).thenReturn(""); + Mockito.when(mConfig.getString("partitioner.granularity.date.format", "yyyy-MM-dd")).thenReturn("'yr='yyyy'/mo='MM'/dy='dd'/hr='HH"); + + assertEquals("yr=2014/mo=07/dy=30/hr=10", new DateMessageParser(mConfig).extractPartitions(mFormat1)[0]); } } diff --git a/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java b/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java new file mode 100644 index 000000000..67233a87c --- /dev/null +++ b/src/test/java/com/pinterest/secor/parser/Iso8601ParserTest.java @@ -0,0 +1,116 @@ +/** + * 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 junit.framework.TestCase; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.TimeZone; + +@RunWith(PowerMockRunner.class) +public class Iso8601ParserTest extends TestCase { + + private SecorConfig mConfig; + private Message mFormat1; + private Message mFormat2; + private Message mFormat3; + private Message mFormat4; + private Message mInvalidDate; + private Message mNestedISOFormat; + private Message mNanosecondISOFormat; + private Message mMissingDate; + private long timestamp; + + @Override + public void setUp() throws Exception { + mConfig = Mockito.mock(SecorConfig.class); + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + Mockito.when(mConfig.getFinalizerDelaySeconds()).thenReturn(3600); + Mockito.when(mConfig.getTimeZone()).thenReturn(TimeZone.getTimeZone("UTC")); + + Mockito.when(TimestampedMessageParser.usingDateFormat(mConfig)).thenReturn("yyyy-MM-dd"); + Mockito.when(TimestampedMessageParser.usingHourFormat(mConfig)).thenReturn("HH"); + Mockito.when(TimestampedMessageParser.usingMinuteFormat(mConfig)).thenReturn("mm"); + Mockito.when(TimestampedMessageParser.usingDatePrefix(mConfig)).thenReturn("dt="); + Mockito.when(TimestampedMessageParser.usingHourPrefix(mConfig)).thenReturn("hr="); + Mockito.when(TimestampedMessageParser.usingMinutePrefix(mConfig)).thenReturn("min="); + + timestamp = System.currentTimeMillis(); + + byte format1[] = "{\"timestamp\":\"2014-07-30T10:53:20.001Z\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}" + .getBytes("UTF-8"); + mFormat1 = new Message("test", 0, 0, null, format1, timestamp); + + byte format2[] = "{\"timestamp\":\"2014-07-29T10:53:20Z\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}" + .getBytes("UTF-8"); + mFormat2 = new Message("test", 0, 0, null, format2, timestamp); + + byte format3[] = "{\"timestamp\":\"2001-07-04Z\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}" + .getBytes("UTF-8"); + mFormat3 = new Message("test", 0, 0, null, format3, timestamp); + + byte format4[] = "{\"timestamp\":\"2016-03-02T18:36:14+00:00\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}" + .getBytes("UTF-8"); + mFormat4 = new Message("test", 0, 0, null, format4, timestamp); + + byte nestedISOFormat[] = "{\"meta_data\":{\"created\":\"2016-01-11T11:50:28.647Z\"},\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}" + .getBytes("UTF-8"); + mNestedISOFormat = new Message("test", 0, 0, null, nestedISOFormat, timestamp); + + byte nanosecondISOFormat[] = "{\"timestamp\":\"2006-01-02T23:59:59.999999999Z\"}" + .getBytes("UTF-8"); + mNanosecondISOFormat = new Message("test", 0, 0, null, nanosecondISOFormat, timestamp); + + byte invalidDate[] = "{\"timestamp\":\"111-11111111\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}" + .getBytes("UTF-8"); + mInvalidDate = new Message("test", 0, 0, null, invalidDate, timestamp); + + byte missingDate[] = "{\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}" + .getBytes("UTF-8"); + mMissingDate = new Message("test", 0, 0, null, missingDate, timestamp); + } + + @Test + public void testExtractTimestampMillis() throws Exception { + Iso8601MessageParser parser = new Iso8601MessageParser(mConfig); + + assertEquals(1406717600001l, parser.getTimestampMillis(mFormat1)); + assertEquals(1406631200000l, parser.getTimestampMillis(mFormat2)); + assertEquals(994204800000l, parser.getTimestampMillis(mFormat3)); + assertEquals(1456943774000l, parser.getTimestampMillis(mFormat4)); + assertEquals(1136246399999l, parser.getTimestampMillis(mNanosecondISOFormat)); + + // Return 0 if there's no timestamp, for any reason. + assertEquals(0l, parser.getTimestampMillis(mInvalidDate)); + assertEquals(0l, parser.getTimestampMillis(mMissingDate)); + } + + @Test + public void testExtractNestedTimestampMillis() throws Exception { + Mockito.when(mConfig.getMessageTimestampNameSeparator()).thenReturn("."); + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("meta_data.created"); + + Iso8601MessageParser parser = new Iso8601MessageParser(mConfig); + + assertEquals(1452513028647l, parser.getTimestampMillis(mNestedISOFormat)); + } +} diff --git a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java index b288c8e55..cdff37f55 100644 --- a/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java +++ b/src/test/java/com/pinterest/secor/parser/JsonMessageParserTest.java @@ -16,7 +16,7 @@ */ package com.pinterest.secor.parser; -import com.pinterest.secor.common.*; +import com.pinterest.secor.common.SecorConfig; import com.pinterest.secor.message.Message; import junit.framework.TestCase; import org.junit.Test; @@ -24,6 +24,11 @@ import org.mockito.Mockito; import org.powermock.modules.junit4.PowerMockRunner; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.TimeZone; + @RunWith(PowerMockRunner.class) public class JsonMessageParserTest extends TestCase { @@ -32,40 +37,76 @@ public class JsonMessageParserTest extends TestCase { private Message mMessageWithMillisTimestamp; private Message mMessageWithMillisFloatTimestamp; private Message mMessageWithoutTimestamp; + private Message mMessageWithNestedTimestamp; + private long timestamp; @Override public void setUp() throws Exception { mConfig = Mockito.mock(SecorConfig.class); Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + Mockito.when(mConfig.getFinalizerDelaySeconds()).thenReturn(3600); + Mockito.when(mConfig.getTimeZone()).thenReturn(TimeZone.getTimeZone("UTC")); + + Mockito.when(TimestampedMessageParser.usingDateFormat(mConfig)).thenReturn("yyyy-MM-dd"); + Mockito.when(TimestampedMessageParser.usingHourFormat(mConfig)).thenReturn("HH"); + Mockito.when(TimestampedMessageParser.usingMinuteFormat(mConfig)).thenReturn("mm"); + Mockito.when(TimestampedMessageParser.usingDatePrefix(mConfig)).thenReturn("dt="); + Mockito.when(TimestampedMessageParser.usingHourPrefix(mConfig)).thenReturn("hr="); + Mockito.when(TimestampedMessageParser.usingMinutePrefix(mConfig)).thenReturn("min="); - byte messageWithSecondsTimestamp[] = - "{\"timestamp\":\"1405970352\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}".getBytes("UTF-8"); - mMessageWithSecondsTimestamp = new Message("test", 0, 0, messageWithSecondsTimestamp); + timestamp = System.currentTimeMillis(); + + byte messageWithSecondsTimestamp[] = + "{\"timestamp\":\"1405911096\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}".getBytes("UTF-8"); + mMessageWithSecondsTimestamp = new Message("test", 0, 0, null, messageWithSecondsTimestamp, timestamp); byte messageWithMillisTimestamp[] = - "{\"timestamp\":\"1405970352123\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}".getBytes("UTF-8"); - mMessageWithMillisTimestamp = new Message("test", 0, 0, messageWithMillisTimestamp); + "{\"timestamp\":\"1405911096123\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}".getBytes("UTF-8"); + mMessageWithMillisTimestamp = new Message("test", 0, 0, null, messageWithMillisTimestamp, timestamp); byte messageWithMillisFloatTimestamp[] = - "{\"timestamp\":\"1405970352123.0\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}".getBytes("UTF-8"); - mMessageWithMillisFloatTimestamp = new Message("test", 0, 0, messageWithMillisFloatTimestamp); + "{\"timestamp\":\"1405911096123.0\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}".getBytes("UTF-8"); + mMessageWithMillisFloatTimestamp = new Message("test", 0, 0, null, messageWithMillisFloatTimestamp, timestamp); byte messageWithoutTimestamp[] = "{\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}".getBytes("UTF-8"); - mMessageWithoutTimestamp = new Message("test", 0, 0, messageWithoutTimestamp); + mMessageWithoutTimestamp = new Message("test", 0, 0, null, messageWithoutTimestamp, 0l); + + byte messageWithNestedTimestamp[] = + "{\"meta_data\":{\"created\":\"1405911096123\"},\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}".getBytes("UTF-8"); + mMessageWithNestedTimestamp = new Message("test", 0, 0, null, messageWithNestedTimestamp, timestamp); } + @Test + public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { + Mockito.when(mConfig.useKafkaTimestamp()).thenReturn(true); + JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); + + assertEquals(timestamp, jsonMessageParser.getTimestampMillis(mMessageWithSecondsTimestamp)); + assertEquals(timestamp, jsonMessageParser.getTimestampMillis(mMessageWithMillisTimestamp)); + assertEquals(timestamp, jsonMessageParser.getTimestampMillis(mMessageWithMillisFloatTimestamp)); + } + @Test public void testExtractTimestampMillis() throws Exception { JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); - assertEquals(1405970352000l, jsonMessageParser.extractTimestampMillis(mMessageWithSecondsTimestamp)); - assertEquals(1405970352123l, jsonMessageParser.extractTimestampMillis(mMessageWithMillisTimestamp)); - assertEquals(1405970352123l, jsonMessageParser.extractTimestampMillis(mMessageWithMillisFloatTimestamp)); + assertEquals(1405911096000l, jsonMessageParser.getTimestampMillis(mMessageWithSecondsTimestamp)); + assertEquals(1405911096123l, jsonMessageParser.getTimestampMillis(mMessageWithMillisTimestamp)); + assertEquals(1405911096123l, jsonMessageParser.getTimestampMillis(mMessageWithMillisFloatTimestamp)); // Return 0 if there's no timestamp, for any reason. - assertEquals(0l, jsonMessageParser.extractTimestampMillis(mMessageWithoutTimestamp)); + assertEquals(0l, jsonMessageParser.getTimestampMillis(mMessageWithoutTimestamp)); + } + + @Test + public void testExtractNestedTimestampMillis() throws Exception { + Mockito.when(mConfig.getMessageTimestampNameSeparator()).thenReturn("."); + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("meta_data.created"); + + JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); + assertEquals(1405911096123l, jsonMessageParser.getTimestampMillis(mMessageWithNestedTimestamp)); } @Test(expected=ClassCastException.class) @@ -73,7 +114,7 @@ public void testExtractTimestampMillisException1() throws Exception { JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); byte emptyBytes1[] = {}; - jsonMessageParser.extractTimestampMillis(new Message("test", 0, 0, emptyBytes1)); + jsonMessageParser.getTimestampMillis(new Message("test", 0, 0, null, emptyBytes1, timestamp)); } @Test(expected=ClassCastException.class) @@ -81,7 +122,7 @@ public void testExtractTimestampMillisException2() throws Exception { JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); byte emptyBytes2[] = "".getBytes(); - jsonMessageParser.extractTimestampMillis(new Message("test", 0, 0, emptyBytes2)); + jsonMessageParser.getTimestampMillis(new Message("test", 0, 0, null, emptyBytes2, timestamp)); } @Test @@ -98,4 +139,302 @@ public void testExtractPartitions() throws Exception { assertEquals(1, resultMillis.length); assertEquals(expectedPartition, resultMillis[0]); } + + @Test + public void testExtractPartitionsForUTCDefaultTimezone() throws Exception { + Mockito.when(mConfig.getTimeZone()).thenReturn(TimeZone.getTimeZone("UTC")); + JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); + + String expectedPartition = "dt=2014-07-21"; + + String resultSeconds[] = jsonMessageParser.extractPartitions(mMessageWithSecondsTimestamp); + assertEquals(1, resultSeconds.length); + assertEquals(expectedPartition, resultSeconds[0]); + + String resultMillis[] = jsonMessageParser.extractPartitions(mMessageWithMillisTimestamp); + assertEquals(1, resultMillis.length); + assertEquals(expectedPartition, resultMillis[0]); + } + + + @Test + public void testExtractHourlyPartitions() throws Exception { + Mockito.when(TimestampedMessageParser.usingHourly(mConfig)).thenReturn(true); + JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); + + String expectedDtPartition = "dt=2014-07-21"; + String expectedHrPartition = "hr=02"; + + String resultSeconds[] = jsonMessageParser.extractPartitions(mMessageWithSecondsTimestamp); + assertEquals(2, resultSeconds.length); + assertEquals(expectedDtPartition, resultSeconds[0]); + assertEquals(expectedHrPartition, resultSeconds[1]); + + String resultMillis[] = jsonMessageParser.extractPartitions(mMessageWithMillisTimestamp); + assertEquals(2, resultMillis.length); + assertEquals(expectedDtPartition, resultMillis[0]); + assertEquals(expectedHrPartition, resultMillis[1]); + } + + @Test + public void testExtractHourlyPartitionsForNonUTCTimezone() throws Exception { + Mockito.when(mConfig.getTimeZone()).thenReturn(TimeZone.getTimeZone("IST")); + Mockito.when(TimestampedMessageParser.usingHourly(mConfig)).thenReturn(true); + JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); + + String expectedDtPartition = "dt=2014-07-21"; + String expectedHrPartition = "hr=08"; + + String resultSeconds[] = jsonMessageParser.extractPartitions(mMessageWithSecondsTimestamp); + assertEquals(2, resultSeconds.length); + assertEquals(expectedDtPartition, resultSeconds[0]); + assertEquals(expectedHrPartition, resultSeconds[1]); + + String resultMillis[] = jsonMessageParser.extractPartitions(mMessageWithMillisTimestamp); + assertEquals(2, resultMillis.length); + assertEquals(expectedDtPartition, resultMillis[0]); + assertEquals(expectedHrPartition, resultMillis[1]); + } + + @Test + public void testDailyGetFinalizedUptoPartitions() throws Exception { + JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); + + List lastMessages = new ArrayList(); + lastMessages.add(mMessageWithSecondsTimestamp); + List committedMessages = new ArrayList(); + committedMessages.add(mMessageWithMillisTimestamp); + String uptoPartitions[] = jsonMessageParser.getFinalizedUptoPartitions(lastMessages, + committedMessages); + assertEquals(1, uptoPartitions.length); + assertEquals("dt=2014-07-21", uptoPartitions[0]); + + String[] previous = jsonMessageParser.getPreviousPartitions(uptoPartitions); + assertEquals(1, previous.length); + assertEquals("dt=2014-07-20", previous[0]); + } + + @Test + public void testHourlyGetFinalizedUptoPartitions() throws Exception { + Mockito.when(TimestampedMessageParser.usingHourly(mConfig)).thenReturn(true); + JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); + + List lastMessages = new ArrayList(); + lastMessages.add(mMessageWithSecondsTimestamp); + List committedMessages = new ArrayList(); + committedMessages.add(mMessageWithMillisTimestamp); + String uptoPartitions[] = jsonMessageParser.getFinalizedUptoPartitions(lastMessages, + committedMessages); + assertEquals(2, uptoPartitions.length); + assertEquals("dt=2014-07-21", uptoPartitions[0]); + assertEquals("hr=01", uptoPartitions[1]); + + String[][] expectedPartitions = new String[][] { + new String[]{"dt=2014-07-21", "hr=00"}, + new String[]{"dt=2014-07-20"}, // there is day partition for previous day + new String[]{"dt=2014-07-20", "hr=23"}, + new String[]{"dt=2014-07-20", "hr=22"}, + new String[]{"dt=2014-07-20", "hr=21"}, + new String[]{"dt=2014-07-20", "hr=20"}, + new String[]{"dt=2014-07-20", "hr=19"}, + new String[]{"dt=2014-07-20", "hr=18"}, + new String[]{"dt=2014-07-20", "hr=17"}, + new String[]{"dt=2014-07-20", "hr=16"}, + new String[]{"dt=2014-07-20", "hr=15"}, + new String[]{"dt=2014-07-20", "hr=14"}, + new String[]{"dt=2014-07-20", "hr=13"}, + new String[]{"dt=2014-07-20", "hr=12"}, + new String[]{"dt=2014-07-20", "hr=11"}, + new String[]{"dt=2014-07-20", "hr=10"}, + new String[]{"dt=2014-07-20", "hr=09"}, + new String[]{"dt=2014-07-20", "hr=08"}, + new String[]{"dt=2014-07-20", "hr=07"}, + new String[]{"dt=2014-07-20", "hr=06"}, + new String[]{"dt=2014-07-20", "hr=05"}, + new String[]{"dt=2014-07-20", "hr=04"}, + new String[]{"dt=2014-07-20", "hr=03"}, + new String[]{"dt=2014-07-20", "hr=02"}, + new String[]{"dt=2014-07-20", "hr=01"}, + new String[]{"dt=2014-07-20", "hr=00"}, + new String[]{"dt=2014-07-19"}, // there is day partition for 2nd last day + new String[]{"dt=2014-07-19", "hr=23"} + }; + + String[] partitions = uptoPartitions; + List partitionsList = new ArrayList(); + for (int i = 0; i < 28; i++ ) { + String[] previous = jsonMessageParser.getPreviousPartitions(partitions); + partitionsList.add(previous); + partitions = previous; + } + + assertEquals(partitionsList.size(), expectedPartitions.length); + for (int i = 0; i < partitionsList.size(); i++) { + List expectedPartition = Arrays.asList(expectedPartitions[i]); + List retrievedPartition = Arrays.asList(partitionsList.get(i)); + assertEquals(expectedPartition, retrievedPartition); + } + } + + @Test + public void testMinutelyGetFinalizedUptoPartitions() throws Exception { + Mockito.when(TimestampedMessageParser.usingMinutely(mConfig)).thenReturn(true); + JsonMessageParser jsonMessageParser = new JsonMessageParser(mConfig); + + List lastMessages = new ArrayList(); + lastMessages.add(mMessageWithSecondsTimestamp); + List committedMessages = new ArrayList(); + committedMessages.add(mMessageWithMillisTimestamp); + String uptoPartitions[] = jsonMessageParser.getFinalizedUptoPartitions(lastMessages, committedMessages); + assertEquals(3, uptoPartitions.length); + assertEquals("dt=2014-07-21", uptoPartitions[0]); + assertEquals("hr=01", uptoPartitions[1]); + assertEquals("min=51", uptoPartitions[2]); + + uptoPartitions[1] = "hr=01"; + uptoPartitions[2] = "min=00"; + uptoPartitions[0] = "dt=2014-07-20"; + + String[][] expectedPartitions = new String[][] { + new String[] { "dt=2014-07-20", "hr=00"}, + new String[] { "dt=2014-07-20", "hr=00", "min=59" }, + new String[] { "dt=2014-07-20", "hr=00", "min=58" }, + new String[] { "dt=2014-07-20", "hr=00", "min=57" }, + new String[] { "dt=2014-07-20", "hr=00", "min=56" }, + new String[] { "dt=2014-07-20", "hr=00", "min=55" }, + new String[] { "dt=2014-07-20", "hr=00", "min=54" }, + new String[] { "dt=2014-07-20", "hr=00", "min=53" }, + new String[] { "dt=2014-07-20", "hr=00", "min=52" }, + new String[] { "dt=2014-07-20", "hr=00", "min=51" }, + new String[] { "dt=2014-07-20", "hr=00", "min=50" }, + new String[] { "dt=2014-07-20", "hr=00", "min=49" }, + new String[] { "dt=2014-07-20", "hr=00", "min=48" }, + new String[] { "dt=2014-07-20", "hr=00", "min=47" }, + new String[] { "dt=2014-07-20", "hr=00", "min=46" }, + new String[] { "dt=2014-07-20", "hr=00", "min=45" }, + new String[] { "dt=2014-07-20", "hr=00", "min=44" }, + new String[] { "dt=2014-07-20", "hr=00", "min=43" }, + new String[] { "dt=2014-07-20", "hr=00", "min=42" }, + new String[] { "dt=2014-07-20", "hr=00", "min=41" }, + new String[] { "dt=2014-07-20", "hr=00", "min=40" }, + new String[] { "dt=2014-07-20", "hr=00", "min=39" }, + new String[] { "dt=2014-07-20", "hr=00", "min=38" }, + new String[] { "dt=2014-07-20", "hr=00", "min=37" }, + new String[] { "dt=2014-07-20", "hr=00", "min=36" }, + new String[] { "dt=2014-07-20", "hr=00", "min=35" }, + new String[] { "dt=2014-07-20", "hr=00", "min=34" }, + new String[] { "dt=2014-07-20", "hr=00", "min=33" }, + new String[] { "dt=2014-07-20", "hr=00", "min=32" }, + new String[] { "dt=2014-07-20", "hr=00", "min=31" }, + new String[] { "dt=2014-07-20", "hr=00", "min=30" }, + new String[] { "dt=2014-07-20", "hr=00", "min=29" }, + new String[] { "dt=2014-07-20", "hr=00", "min=28" }, + new String[] { "dt=2014-07-20", "hr=00", "min=27" }, + new String[] { "dt=2014-07-20", "hr=00", "min=26" }, + new String[] { "dt=2014-07-20", "hr=00", "min=25" }, + new String[] { "dt=2014-07-20", "hr=00", "min=24" }, + new String[] { "dt=2014-07-20", "hr=00", "min=23" }, + new String[] { "dt=2014-07-20", "hr=00", "min=22" }, + new String[] { "dt=2014-07-20", "hr=00", "min=21" }, + new String[] { "dt=2014-07-20", "hr=00", "min=20" }, + new String[] { "dt=2014-07-20", "hr=00", "min=19" }, + new String[] { "dt=2014-07-20", "hr=00", "min=18" }, + new String[] { "dt=2014-07-20", "hr=00", "min=17" }, + new String[] { "dt=2014-07-20", "hr=00", "min=16" }, + new String[] { "dt=2014-07-20", "hr=00", "min=15" }, + new String[] { "dt=2014-07-20", "hr=00", "min=14" }, + new String[] { "dt=2014-07-20", "hr=00", "min=13" }, + new String[] { "dt=2014-07-20", "hr=00", "min=12" }, + new String[] { "dt=2014-07-20", "hr=00", "min=11" }, + new String[] { "dt=2014-07-20", "hr=00", "min=10" }, + new String[] { "dt=2014-07-20", "hr=00", "min=09" }, + new String[] { "dt=2014-07-20", "hr=00", "min=08" }, + new String[] { "dt=2014-07-20", "hr=00", "min=07" }, + new String[] { "dt=2014-07-20", "hr=00", "min=06" }, + new String[] { "dt=2014-07-20", "hr=00", "min=05" }, + new String[] { "dt=2014-07-20", "hr=00", "min=04" }, + new String[] { "dt=2014-07-20", "hr=00", "min=03" }, + new String[] { "dt=2014-07-20", "hr=00", "min=02" }, + new String[] { "dt=2014-07-20", "hr=00", "min=01" }, + new String[] { "dt=2014-07-20", "hr=00", "min=00" }, + new String[] { "dt=2014-07-19" }, + new String[] { "dt=2014-07-19", "hr=23"}, + new String[] { "dt=2014-07-19", "hr=23", "min=59" }, + new String[] { "dt=2014-07-19", "hr=23", "min=58" }, + new String[] { "dt=2014-07-19", "hr=23", "min=57" }, + new String[] { "dt=2014-07-19", "hr=23", "min=56" }, + new String[] { "dt=2014-07-19", "hr=23", "min=55" }, + new String[] { "dt=2014-07-19", "hr=23", "min=54" }, + new String[] { "dt=2014-07-19", "hr=23", "min=53" }, + new String[] { "dt=2014-07-19", "hr=23", "min=52" }, + new String[] { "dt=2014-07-19", "hr=23", "min=51" }, + new String[] { "dt=2014-07-19", "hr=23", "min=50" }, + new String[] { "dt=2014-07-19", "hr=23", "min=49" }, + new String[] { "dt=2014-07-19", "hr=23", "min=48" }, + new String[] { "dt=2014-07-19", "hr=23", "min=47" }, + new String[] { "dt=2014-07-19", "hr=23", "min=46" }, + new String[] { "dt=2014-07-19", "hr=23", "min=45" }, + new String[] { "dt=2014-07-19", "hr=23", "min=44" }, + new String[] { "dt=2014-07-19", "hr=23", "min=43" }, + new String[] { "dt=2014-07-19", "hr=23", "min=42" }, + new String[] { "dt=2014-07-19", "hr=23", "min=41" }, + new String[] { "dt=2014-07-19", "hr=23", "min=40" }, + new String[] { "dt=2014-07-19", "hr=23", "min=39" }, + new String[] { "dt=2014-07-19", "hr=23", "min=38" }, + new String[] { "dt=2014-07-19", "hr=23", "min=37" }, + new String[] { "dt=2014-07-19", "hr=23", "min=36" }, + new String[] { "dt=2014-07-19", "hr=23", "min=35" }, + new String[] { "dt=2014-07-19", "hr=23", "min=34" }, + new String[] { "dt=2014-07-19", "hr=23", "min=33" }, + new String[] { "dt=2014-07-19", "hr=23", "min=32" }, + new String[] { "dt=2014-07-19", "hr=23", "min=31" }, + new String[] { "dt=2014-07-19", "hr=23", "min=30" }, + new String[] { "dt=2014-07-19", "hr=23", "min=29" }, + new String[] { "dt=2014-07-19", "hr=23", "min=28" }, + new String[] { "dt=2014-07-19", "hr=23", "min=27" }, + new String[] { "dt=2014-07-19", "hr=23", "min=26" }, + new String[] { "dt=2014-07-19", "hr=23", "min=25" }, + new String[] { "dt=2014-07-19", "hr=23", "min=24" }, + new String[] { "dt=2014-07-19", "hr=23", "min=23" }, + new String[] { "dt=2014-07-19", "hr=23", "min=22" }, + new String[] { "dt=2014-07-19", "hr=23", "min=21" }, + new String[] { "dt=2014-07-19", "hr=23", "min=20" }, + new String[] { "dt=2014-07-19", "hr=23", "min=19" }, + new String[] { "dt=2014-07-19", "hr=23", "min=18" }, + new String[] { "dt=2014-07-19", "hr=23", "min=17" }, + new String[] { "dt=2014-07-19", "hr=23", "min=16" }, + new String[] { "dt=2014-07-19", "hr=23", "min=15" }, + new String[] { "dt=2014-07-19", "hr=23", "min=14" }, + new String[] { "dt=2014-07-19", "hr=23", "min=13" }, + new String[] { "dt=2014-07-19", "hr=23", "min=12" }, + new String[] { "dt=2014-07-19", "hr=23", "min=11" }, + new String[] { "dt=2014-07-19", "hr=23", "min=10" }, + new String[] { "dt=2014-07-19", "hr=23", "min=09" }, + new String[] { "dt=2014-07-19", "hr=23", "min=08" }, + new String[] { "dt=2014-07-19", "hr=23", "min=07" }, + new String[] { "dt=2014-07-19", "hr=23", "min=06" }, + new String[] { "dt=2014-07-19", "hr=23", "min=05" }, + new String[] { "dt=2014-07-19", "hr=23", "min=04" }, + new String[] { "dt=2014-07-19", "hr=23", "min=03" }, + new String[] { "dt=2014-07-19", "hr=23", "min=02" }, + new String[] { "dt=2014-07-19", "hr=23", "min=01" }, + new String[] { "dt=2014-07-19", "hr=23", "min=00" }, + new String[] { "dt=2014-07-19", "hr=22" }, + new String[] { "dt=2014-07-19", "hr=22", "min=59" }, }; + + String[] partitions = uptoPartitions; + List partitionsList = new ArrayList(); + for (int i = 0; i < 125; i++) { + String[] previous = jsonMessageParser.getPreviousPartitions(partitions); + partitionsList.add(previous); + partitions = previous; + } + + assertEquals(partitionsList.size(), expectedPartitions.length); + for (int i = 0; i < partitionsList.size(); i++) { + List expectedPartition = Arrays.asList(expectedPartitions[i]); + List retrievedPartition = Arrays.asList(partitionsList.get(i)); + assertEquals(expectedPartition, retrievedPartition); + } + } } diff --git a/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java new file mode 100644 index 000000000..70c6850c7 --- /dev/null +++ b/src/test/java/com/pinterest/secor/parser/MessagePackParserTest.java @@ -0,0 +1,159 @@ +/** + * 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.fasterxml.jackson.databind.ObjectMapper; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; +import junit.framework.TestCase; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.msgpack.jackson.dataformat.MessagePackFactory; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.HashMap; +import java.util.TimeZone; + +@RunWith(PowerMockRunner.class) +public class MessagePackParserTest extends TestCase { + + SecorConfig mConfig; + private MessagePackParser mMessagePackParser; + private Message mMessageWithSecondsTimestamp; + private Message mMessageWithMillisTimestamp; + private Message mMessageWithMillisFloatTimestamp; + private Message mMessageWithMillisStringTimestamp; + private ObjectMapper mObjectMapper; + private long timestamp; + + @Override + public void setUp() throws Exception { + mConfig = Mockito.mock(SecorConfig.class); + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("ts"); + Mockito.when(mConfig.getTimeZone()).thenReturn(TimeZone.getTimeZone("UTC")); + Mockito.when(TimestampedMessageParser.usingDateFormat(mConfig)).thenReturn("yyyy-MM-dd"); + Mockito.when(TimestampedMessageParser.usingHourFormat(mConfig)).thenReturn("HH"); + Mockito.when(TimestampedMessageParser.usingMinuteFormat(mConfig)).thenReturn("mm"); + Mockito.when(TimestampedMessageParser.usingDatePrefix(mConfig)).thenReturn("dt="); + Mockito.when(TimestampedMessageParser.usingHourPrefix(mConfig)).thenReturn("hr="); + Mockito.when(TimestampedMessageParser.usingMinutePrefix(mConfig)).thenReturn("min="); + + mMessagePackParser = new MessagePackParser(mConfig); + mObjectMapper = new ObjectMapper(new MessagePackFactory()); + + timestamp = System.currentTimeMillis(); + + HashMap mapWithSecondTimestamp = new HashMap(); + mapWithSecondTimestamp.put("ts", 1405970352); + mMessageWithSecondsTimestamp = new Message("test", 0, 0, null, + mObjectMapper.writeValueAsBytes(mapWithSecondTimestamp), timestamp); + + HashMap mapWithMillisTimestamp = new HashMap(); + mapWithMillisTimestamp.put("ts", 1405970352123l); + mapWithMillisTimestamp.put("isActive", true); + mapWithMillisTimestamp.put("email", "alice@example.com"); + mapWithMillisTimestamp.put("age", 27); + mMessageWithMillisTimestamp = new Message("test", 0, 0, null, + mObjectMapper.writeValueAsBytes(mapWithMillisTimestamp), timestamp); + + + HashMap mapWithMillisFloatTimestamp = new HashMap(); + mapWithMillisFloatTimestamp.put("ts", 1405970352123.0); + mapWithMillisFloatTimestamp.put("isActive", false); + mapWithMillisFloatTimestamp.put("email", "bob@example.com"); + mapWithMillisFloatTimestamp.put("age", 35); + mMessageWithMillisFloatTimestamp = new Message("test", 0, 0, null, + mObjectMapper.writeValueAsBytes(mapWithMillisFloatTimestamp), timestamp); + + HashMap mapWithMillisStringTimestamp = new HashMap(); + mapWithMillisStringTimestamp.put("ts", "1405970352123"); + mapWithMillisStringTimestamp.put("isActive", null); + mapWithMillisStringTimestamp.put("email", "charlie@example.com"); + mapWithMillisStringTimestamp.put("age", 67); + mMessageWithMillisStringTimestamp = new Message("test", 0, 0, null, + mObjectMapper.writeValueAsBytes(mapWithMillisStringTimestamp), timestamp); + } + + @Test + public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { + Mockito.when(mConfig.useKafkaTimestamp()).thenReturn(true); + mMessagePackParser = new MessagePackParser(mConfig); + + assertEquals(timestamp, mMessagePackParser.getTimestampMillis( + mMessageWithSecondsTimestamp)); + assertEquals(timestamp, mMessagePackParser.getTimestampMillis( + mMessageWithMillisTimestamp)); + assertEquals(timestamp, mMessagePackParser.getTimestampMillis( + mMessageWithMillisFloatTimestamp)); + assertEquals(timestamp, mMessagePackParser.getTimestampMillis( + mMessageWithMillisStringTimestamp)); + } + + @Test + public void testExtractTimestampMillis() throws Exception { + assertEquals(1405970352000l, mMessagePackParser.getTimestampMillis( + mMessageWithSecondsTimestamp)); + assertEquals(1405970352123l, mMessagePackParser.getTimestampMillis( + mMessageWithMillisTimestamp)); + assertEquals(1405970352123l, mMessagePackParser.getTimestampMillis( + mMessageWithMillisFloatTimestamp)); + assertEquals(1405970352123l, mMessagePackParser.getTimestampMillis( + mMessageWithMillisStringTimestamp)); + } + + @Test(expected=NullPointerException.class) + public void testMissingTimestamp() throws Exception { + HashMap mapWithoutTimestamp = new HashMap(); + mapWithoutTimestamp.put("email", "mary@example.com"); + Message nMessageWithoutTimestamp = new Message("test", 0, 0, null, + mObjectMapper.writeValueAsBytes(mapWithoutTimestamp), timestamp); + mMessagePackParser.getTimestampMillis(nMessageWithoutTimestamp); + } + + @Test(expected=NumberFormatException.class) + public void testUnsupportedTimestampFormat() throws Exception { + HashMap mapWitUnsupportedFormatTimestamp = new HashMap(); + mapWitUnsupportedFormatTimestamp.put("ts", "2014-11-14T18:12:52.878Z"); + Message nMessageWithUnsupportedFormatTimestamp = new Message("test", 0, 0, null, + mObjectMapper.writeValueAsBytes(mapWitUnsupportedFormatTimestamp), timestamp); + mMessagePackParser.getTimestampMillis(nMessageWithUnsupportedFormatTimestamp); + } + + @Test(expected=NullPointerException.class) + public void testNullTimestamp() throws Exception { + HashMap mapWitNullTimestamp = new HashMap(); + mapWitNullTimestamp.put("ts", null); + Message nMessageWithNullTimestamp = new Message("test", 0, 0, null, + mObjectMapper.writeValueAsBytes(mapWitNullTimestamp), timestamp); + mMessagePackParser.getTimestampMillis(nMessageWithNullTimestamp); + } + + @Test + public void testExtractPartitions() throws Exception { + String expectedPartition = "dt=2014-07-21"; + + String resultSeconds[] = mMessagePackParser.extractPartitions(mMessageWithSecondsTimestamp); + assertEquals(1, resultSeconds.length); + assertEquals(expectedPartition, resultSeconds[0]); + + String resultMillis[] = mMessagePackParser.extractPartitions(mMessageWithMillisTimestamp); + assertEquals(1, resultMillis.length); + assertEquals(expectedPartition, resultMillis[0]); + } +} diff --git a/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.java new file mode 100644 index 000000000..ce92dc147 --- /dev/null +++ b/src/test/java/com/pinterest/secor/parser/ProtobufMessageParserTest.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.parser; + +import java.util.HashMap; +import java.util.Map; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.modules.junit4.PowerMockRunner; + +import com.google.protobuf.CodedOutputStream; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; +import com.pinterest.secor.protobuf.Messages.UnitTestMessage1; +import com.pinterest.secor.protobuf.Messages.UnitTestMessage2; + +import junit.framework.TestCase; + +@RunWith(PowerMockRunner.class) +public class ProtobufMessageParserTest extends TestCase { + private SecorConfig mConfig; + private long timestamp; + + private Message buildMessage(long timestamp) throws Exception { + byte data[] = new byte[16]; + CodedOutputStream output = CodedOutputStream.newInstance(data); + output.writeUInt64(1, timestamp); + return new Message("test", 0, 0, null, data, timestamp); + } + + @Override + public void setUp() throws Exception { + mConfig = Mockito.mock(SecorConfig.class); + Mockito.when(TimestampedMessageParser.usingDateFormat(mConfig)).thenReturn("yyyy-MM-dd"); + Mockito.when(TimestampedMessageParser.usingHourFormat(mConfig)).thenReturn("HH"); + Mockito.when(TimestampedMessageParser.usingMinuteFormat(mConfig)).thenReturn("mm"); + Mockito.when(TimestampedMessageParser.usingDatePrefix(mConfig)).thenReturn("dt="); + Mockito.when(TimestampedMessageParser.usingHourPrefix(mConfig)).thenReturn("hr="); + Mockito.when(TimestampedMessageParser.usingMinutePrefix(mConfig)).thenReturn("min="); + + timestamp = System.currentTimeMillis(); + } + + @Test + public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { + Mockito.when(mConfig.getBoolean("kafka.useTimestamp", false)).thenReturn(true); + ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); + + assertEquals(1405970352000l, parser.extractTimestampMillis(buildMessage(1405970352l))); + assertEquals(1405970352123l, parser.extractTimestampMillis(buildMessage(1405970352123l))); + } + + @Test + public void testExtractTimestampMillis() throws Exception { + ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); + + assertEquals(1405970352000l, parser.extractTimestampMillis(buildMessage(1405970352l))); + assertEquals(1405970352123l, parser.extractTimestampMillis(buildMessage(1405970352123l))); + } + + @Test + public void testExtractPathTimestampMillis() throws Exception { + Map classPerTopic = new HashMap(); + classPerTopic.put("test", UnitTestMessage1.class.getName()); + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + Mockito.when(mConfig.getProtobufMessageClassPerTopic()).thenReturn(classPerTopic); + + ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); + + UnitTestMessage1 message = UnitTestMessage1.newBuilder().setTimestamp(1405970352L).build(); + assertEquals(1405970352000l, + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray(), timestamp))); + + message = UnitTestMessage1.newBuilder().setTimestamp(1405970352123l).build(); + assertEquals(1405970352123l, + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray(), timestamp))); + } + + @Test + public void testExtractNestedTimestampMillis() throws Exception { + Map classPerTopic = new HashMap(); + classPerTopic.put("*", UnitTestMessage2.class.getName()); + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("internal.timestamp"); + Mockito.when(mConfig.getProtobufMessageClassPerTopic()).thenReturn(classPerTopic); + + ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); + + UnitTestMessage2 message = UnitTestMessage2.newBuilder() + .setInternal(UnitTestMessage2.Internal.newBuilder().setTimestamp(1405970352L).build()).build(); + assertEquals(1405970352000l, + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray(), timestamp))); + + message = UnitTestMessage2.newBuilder() + .setInternal(UnitTestMessage2.Internal.newBuilder().setTimestamp(1405970352123l).build()).build(); + assertEquals(1405970352123l, + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray(), timestamp))); + } +} diff --git a/src/test/java/com/pinterest/secor/parser/ProtobufTimestampParserTest.java b/src/test/java/com/pinterest/secor/parser/ProtobufTimestampParserTest.java new file mode 100644 index 000000000..1c662165c --- /dev/null +++ b/src/test/java/com/pinterest/secor/parser/ProtobufTimestampParserTest.java @@ -0,0 +1,111 @@ +package com.pinterest.secor.parser; + +import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.Timestamp; +import com.google.protobuf.util.Timestamps; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; +import com.pinterest.secor.protobuf.Messages; +import com.pinterest.secor.protobuf.TimestampedMessages; +import junit.framework.TestCase; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.HashMap; +import java.util.Map; + +/** + * Created by pgautam on 10/9/16. + */ +@RunWith(PowerMockRunner.class) +public class ProtobufTimestampParserTest extends TestCase { + private SecorConfig mConfig; + private long timestamp; + + private Message buildMessage(long timestamp) throws Exception { + byte data[] = new byte[16]; + CodedOutputStream output = CodedOutputStream.newInstance(data); + output.writeUInt64(1, timestamp); + return new Message("test", 0, 0, null, data, timestamp); + } + + @Override + public void setUp() throws Exception { + mConfig = Mockito.mock(SecorConfig.class); + Mockito.when(TimestampedMessageParser.usingDateFormat(mConfig)).thenReturn("yyyy-MM-dd"); + Mockito.when(TimestampedMessageParser.usingHourFormat(mConfig)).thenReturn("HH"); + Mockito.when(TimestampedMessageParser.usingMinuteFormat(mConfig)).thenReturn("mm"); + Mockito.when(TimestampedMessageParser.usingDatePrefix(mConfig)).thenReturn("dt="); + Mockito.when(TimestampedMessageParser.usingHourPrefix(mConfig)).thenReturn("hr="); + Mockito.when(TimestampedMessageParser.usingMinutePrefix(mConfig)).thenReturn("min="); + + timestamp = System.currentTimeMillis(); + } + + @Test + public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { + Mockito.when(mConfig.getBoolean("kafka.useTimestamp", false)).thenReturn(true); + ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); + + assertEquals(1405970352000l, parser.extractTimestampMillis(buildMessage(1405970352l))); + assertEquals(1405970352123l, parser.extractTimestampMillis(buildMessage(1405970352123l))); + } + + @Test + public void testExtractTimestampMillis() throws Exception { + ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); + + assertEquals(1405970352000l, parser.extractTimestampMillis(buildMessage(1405970352l))); + assertEquals(1405970352123l, parser.extractTimestampMillis(buildMessage(1405970352123l))); + } + + @Test + public void testExtractPathTimestampMillis() throws Exception { + Map classPerTopic = new HashMap(); + System.out.println(TimestampedMessages.UnitTestTimestamp1.class.getName()); + classPerTopic.put("test", TimestampedMessages.UnitTestTimestamp1.class.getName()); + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + Mockito.when(mConfig.getProtobufMessageClassPerTopic()).thenReturn(classPerTopic); + + ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); + + + Timestamp timestamp = Timestamp.newBuilder().setSeconds(1405970352l) + .setNanos(0).build(); + + TimestampedMessages.UnitTestTimestamp1 message = TimestampedMessages.UnitTestTimestamp1.newBuilder().setTimestamp(timestamp).build(); + assertEquals(1405970352000l, + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray(), timestamp.getSeconds()))); + + Timestamp timestampWithNano = Timestamp.newBuilder().setSeconds(1405970352l) + .setNanos(123000000).build(); + message = TimestampedMessages.UnitTestTimestamp1.newBuilder().setTimestamp(timestampWithNano).build(); + assertEquals(1405970352123l, + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray(), timestamp.getSeconds()))); + } + + @Test + public void testExtractNestedTimestampMillis() throws Exception { + Map classPerTopic = new HashMap(); + classPerTopic.put("*", TimestampedMessages.UnitTestTimestamp2.class.getName()); + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("internal.timestamp"); + Mockito.when(mConfig.getProtobufMessageClassPerTopic()).thenReturn(classPerTopic); + + ProtobufMessageParser parser = new ProtobufMessageParser(mConfig); + + Timestamp timestamp = Timestamps.fromMillis(1405970352000L); + + TimestampedMessages.UnitTestTimestamp2 message = TimestampedMessages.UnitTestTimestamp2.newBuilder() + .setInternal(TimestampedMessages.UnitTestTimestamp2.Internal.newBuilder().setTimestamp(timestamp).build()).build(); + assertEquals(1405970352000l, + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray(), timestamp.getSeconds()))); + + timestamp = Timestamps.fromMillis(1405970352123l); + message = TimestampedMessages.UnitTestTimestamp2.newBuilder() + .setInternal(TimestampedMessages.UnitTestTimestamp2.Internal.newBuilder().setTimestamp(timestamp).build()).build(); + assertEquals(1405970352123l, + parser.extractTimestampMillis(new Message("test", 0, 0, null, message.toByteArray(), timestamp.getSeconds()))); + } +} diff --git a/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java new file mode 100644 index 000000000..5f9bcbdab --- /dev/null +++ b/src/test/java/com/pinterest/secor/parser/RegexMessageParserTest.java @@ -0,0 +1,88 @@ +/** + * 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 junit.framework.TestCase; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.modules.junit4.PowerMockRunner; + +@RunWith(PowerMockRunner.class) +public class RegexMessageParserTest extends TestCase { + + private SecorConfig mConfig; + private Message mMessageWithMillisTimestamp; + private Message mMessageWithWrongFormatTimestamp; + private long timestamp; + + @Override + public void setUp() throws Exception { + mConfig = Mockito.mock(SecorConfig.class); + Mockito.when(mConfig.getMessageTimestampInputPattern()).thenReturn("^[^ ]+ [^ ]+ ([^ ]+) .*$"); + + Mockito.when(TimestampedMessageParser.usingDateFormat(mConfig)).thenReturn("yyyy-MM-dd"); + Mockito.when(TimestampedMessageParser.usingHourFormat(mConfig)).thenReturn("HH"); + Mockito.when(TimestampedMessageParser.usingMinuteFormat(mConfig)).thenReturn("mm"); + Mockito.when(TimestampedMessageParser.usingDatePrefix(mConfig)).thenReturn("dt="); + Mockito.when(TimestampedMessageParser.usingHourPrefix(mConfig)).thenReturn("hr="); + Mockito.when(TimestampedMessageParser.usingMinutePrefix(mConfig)).thenReturn("min="); + + timestamp = System.currentTimeMillis(); + + byte messageWithMillisTimestamp[] = + "?24.140.88.218 2015/09/22T22:19:00+0000 1442960340 GET http://api.com/test/?id=123 HTTP/1.1 s200 1017 0.384213448 pass - r685206763364 91ea566f - \"for iOS/5.4.2 (iPhone; 9.0)\"".getBytes("UTF-8"); + mMessageWithMillisTimestamp = new Message("test", 0, 0, null, messageWithMillisTimestamp, timestamp); + + byte messageWithWrongFormatTimestamp[] = + "?24.140.88.218 2015/09/22T22:19:00+0000 A1442960340 GET http://api.com/test/?id=123 HTTP/1.1 s200 1017 0.384213448 pass - r685206763364 91ea566f - \"for iOS/5.4.2 (iPhone; 9.0)\"".getBytes("UTF-8"); + mMessageWithWrongFormatTimestamp = new Message("test", 0, 0, null, messageWithWrongFormatTimestamp, timestamp); + + } + + @Test + public void testExtractTimestampMillisFromKafkaTimestamp() throws Exception { + Mockito.when(mConfig.useKafkaTimestamp()).thenReturn(true); + RegexMessageParser regexMessageParser = new RegexMessageParser(mConfig); + + assertEquals(timestamp, regexMessageParser.getTimestampMillis(mMessageWithMillisTimestamp)); + } + + @Test + public void testExtractTimestampMillis() throws Exception { + RegexMessageParser regexMessageParser = new RegexMessageParser(mConfig); + + assertEquals(1442960340000l, regexMessageParser.extractTimestampMillis(mMessageWithMillisTimestamp)); + } + + @Test(expected=NumberFormatException.class) + public void testExtractTimestampMillisEmpty() throws Exception { + RegexMessageParser regexMessageParser = new RegexMessageParser(mConfig); + byte emptyBytes2[] = "".getBytes(); + regexMessageParser.extractTimestampMillis(new Message("test", 0, 0, null, emptyBytes2, timestamp)); + } + + @Test(expected=NumberFormatException.class) + public void testExtractTimestampMillisException1() throws Exception { + RegexMessageParser regexMessageParser = new RegexMessageParser(mConfig); + regexMessageParser.extractTimestampMillis(mMessageWithWrongFormatTimestamp); + } + +} diff --git a/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java new file mode 100644 index 000000000..705979081 --- /dev/null +++ b/src/test/java/com/pinterest/secor/parser/SplitByFieldMessageParserTest.java @@ -0,0 +1,181 @@ +/** + * 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 junit.framework.TestCase; +import net.minidev.json.JSONObject; +import net.minidev.json.JSONValue; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.ArrayList; +import java.util.List; +import java.util.TimeZone; + +@RunWith(PowerMockRunner.class) +public class SplitByFieldMessageParserTest extends TestCase { + + private SecorConfig mConfig; + private Message mMessageWithTypeAndTimestamp; + private Message mMessageWithoutTimestamp; + private Message mMessageWithoutType; + private long timestamp; + + @Override + public void setUp() throws Exception { + mConfig = Mockito.mock(SecorConfig.class); + Mockito.when(mConfig.getMessageSplitFieldName()).thenReturn("type"); + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestamp"); + Mockito.when(mConfig.getFinalizerDelaySeconds()).thenReturn(3600); + Mockito.when(mConfig.getTimeZone()).thenReturn(TimeZone.getTimeZone("UTC")); + + Mockito.when(TimestampedMessageParser.usingDateFormat(mConfig)).thenReturn("yyyy-MM-dd"); + Mockito.when(TimestampedMessageParser.usingHourFormat(mConfig)).thenReturn("HH"); + Mockito.when(TimestampedMessageParser.usingMinuteFormat(mConfig)).thenReturn("mm"); + Mockito.when(TimestampedMessageParser.usingDatePrefix(mConfig)).thenReturn("dt="); + Mockito.when(TimestampedMessageParser.usingHourPrefix(mConfig)).thenReturn("hr="); + Mockito.when(TimestampedMessageParser.usingMinutePrefix(mConfig)).thenReturn("min="); + + timestamp = System.currentTimeMillis(); + + byte messageWithTypeAndTimestamp[] = + "{\"type\":\"event1\",\"timestamp\":\"1405911096000\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}".getBytes("UTF-8"); + mMessageWithTypeAndTimestamp = new Message("test", 0, 0, null, messageWithTypeAndTimestamp, timestamp); + + byte messageWithoutTimestamp[] = + "{\"type\":\"event2\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}".getBytes("UTF-8"); + mMessageWithoutTimestamp = new Message("test", 0, 0, null, messageWithoutTimestamp, timestamp); + + byte messageWithoutType[] = + "{\"timestamp\":\"1405911096123\",\"id\":0,\"guid\":\"0436b17b-e78a-4e82-accf-743bf1f0b884\",\"isActive\":false,\"balance\":\"$3,561.87\",\"picture\":\"http://placehold.it/32x32\",\"age\":23,\"eyeColor\":\"green\",\"name\":\"Mercedes Brewer\",\"gender\":\"female\",\"company\":\"MALATHION\",\"email\":\"mercedesbrewer@malathion.com\",\"phone\":\"+1 (848) 471-3000\",\"address\":\"786 Gilmore Court, Brule, Maryland, 3200\",\"about\":\"Quis nostrud Lorem deserunt esse ut reprehenderit aliqua nisi et sunt mollit est. Cupidatat incididunt minim anim eiusmod culpa elit est dolor ullamco. Aliqua cillum eiusmod ullamco nostrud Lorem sit amet Lorem aliquip esse esse velit.\\r\\n\",\"registered\":\"2014-01-14T13:07:28 +08:00\",\"latitude\":47.672012,\"longitude\":102.788623,\"tags\":[\"amet\",\"amet\",\"dolore\",\"eu\",\"qui\",\"fugiat\",\"laborum\"],\"friends\":[{\"id\":0,\"name\":\"Rebecca Hardy\"},{\"id\":1,\"name\":\"Sutton Briggs\"},{\"id\":2,\"name\":\"Dena Campos\"}],\"greeting\":\"Hello, Mercedes Brewer! You have 7 unread messages.\",\"favoriteFruit\":\"strawberry\"}".getBytes("UTF-8"); + mMessageWithoutType = new Message("test", 0, 0, null, messageWithoutType, timestamp); + } + + @Test + public void testExtractTypeAndTimestamp() throws Exception { + SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); + + assertEquals(1405911096000l, jsonMessageParser.extractTimestampMillis((JSONObject) JSONValue.parse(mMessageWithTypeAndTimestamp.getPayload()))); + assertEquals(1405911096123l, jsonMessageParser.extractTimestampMillis((JSONObject) JSONValue.parse(mMessageWithoutType.getPayload()))); + + assertEquals("event1", jsonMessageParser.extractEventType((JSONObject) JSONValue.parse(mMessageWithTypeAndTimestamp.getPayload()))); + assertEquals("event2", jsonMessageParser.extractEventType((JSONObject) JSONValue.parse(mMessageWithoutTimestamp.getPayload()))); + } + + @Test(expected = RuntimeException.class) + public void testExtractTimestampMillisExceptionNoTimestamp() throws Exception { + SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); + + // Throws exception if there's no timestamp, for any reason. + jsonMessageParser.extractTimestampMillis((JSONObject) JSONValue.parse(mMessageWithoutTimestamp.getPayload())); + } + + @Test(expected = ClassCastException.class) + public void testExtractTimestampMillisException1() throws Exception { + SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); + + byte emptyBytes1[] = {}; + jsonMessageParser.extractTimestampMillis((JSONObject) JSONValue.parse(emptyBytes1)); + } + + @Test(expected = ClassCastException.class) + public void testExtractTimestampMillisException2() throws Exception { + SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); + + byte emptyBytes2[] = "".getBytes(); + jsonMessageParser.extractTimestampMillis((JSONObject) JSONValue.parse(emptyBytes2)); + } + + @Test(expected = RuntimeException.class) + public void testExtractTimestampMillisExceptionNoType() throws Exception { + SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); + + // Throws exception if there's no timestamp, for any reason. + jsonMessageParser.extractEventType((JSONObject) JSONValue.parse(mMessageWithoutType.getPayload())); + } + + @Test + public void testExtractPartitions() throws Exception { + SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); + + String expectedEventTypePartition = "event1"; + String expectedDtPartition = "dt=2014-07-21"; + + String result[] = jsonMessageParser.extractPartitions(mMessageWithTypeAndTimestamp); + assertEquals(2, result.length); + assertEquals(expectedEventTypePartition, result[0]); + assertEquals(expectedDtPartition, result[1]); + } + + @Test + public void testExtractHourlyPartitions() throws Exception { + Mockito.when(TimestampedMessageParser.usingHourly(mConfig)).thenReturn(true); + SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); + + String expectedEventTypePartition = "event1"; + String expectedDtPartition = "dt=2014-07-21"; + String expectedHrPartition = "hr=02"; + + String result[] = jsonMessageParser.extractPartitions(mMessageWithTypeAndTimestamp); + assertEquals(3, result.length); + assertEquals(expectedEventTypePartition, result[0]); + assertEquals(expectedDtPartition, result[1]); + assertEquals(expectedHrPartition, result[2]); + } + + @Test + public void testExtractHourlyPartitionsForNonUTCTimezone() throws Exception { + Mockito.when(mConfig.getTimeZone()).thenReturn(TimeZone.getTimeZone("IST")); + Mockito.when(TimestampedMessageParser.usingHourly(mConfig)).thenReturn(true); + SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); + + String expectedEventTypePartition = "event1"; + String expectedDtPartition = "dt=2014-07-21"; + String expectedHrPartition = "hr=08"; + + String result[] = jsonMessageParser.extractPartitions(mMessageWithTypeAndTimestamp); + assertEquals(3, result.length); + assertEquals(expectedEventTypePartition, result[0]); + assertEquals(expectedDtPartition, result[1]); + assertEquals(expectedHrPartition, result[2]); + } + + @Test(expected = UnsupportedOperationException.class) + public void testGetFinalizedUptoPartitions() throws Exception { + SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); + + List lastMessages = new ArrayList(); + lastMessages.add(mMessageWithTypeAndTimestamp); + List committedMessages = new ArrayList(); + committedMessages.add(mMessageWithTypeAndTimestamp); + + jsonMessageParser.getFinalizedUptoPartitions(lastMessages, committedMessages); + } + + @Test(expected = UnsupportedOperationException.class) + public void testGetPreviousPartitions() throws Exception { + SplitByFieldMessageParser jsonMessageParser = new SplitByFieldMessageParser(mConfig); + + String partitions[] = {"event1", "dt=2014-07-21"}; + jsonMessageParser.getPreviousPartitions(partitions); + } + +} diff --git a/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java b/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java new file mode 100644 index 000000000..32692d7ce --- /dev/null +++ b/src/test/java/com/pinterest/secor/parser/ThriftMessageParserTest.java @@ -0,0 +1,104 @@ +package com.pinterest.secor.parser; + +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.message.Message; +import junit.framework.TestCase; +import org.apache.thrift.TSerializer; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.modules.junit4.PowerMockRunner; + +import com.pinterest.secor.thrift.UnitTestMessage; + +@RunWith(PowerMockRunner.class) +public class ThriftMessageParserTest extends TestCase { + private SecorConfig mConfig; + private long timestamp; + + @Override + public void setUp() throws Exception { + mConfig = Mockito.mock(SecorConfig.class); + Mockito.when(TimestampedMessageParser.usingDateFormat(mConfig)).thenReturn("yyyy-MM-dd"); + Mockito.when(TimestampedMessageParser.usingHourFormat(mConfig)).thenReturn("HH"); + Mockito.when(TimestampedMessageParser.usingMinuteFormat(mConfig)).thenReturn("mm"); + Mockito.when(TimestampedMessageParser.usingDatePrefix(mConfig)).thenReturn("dt="); + Mockito.when(TimestampedMessageParser.usingHourPrefix(mConfig)).thenReturn("hr="); + Mockito.when(TimestampedMessageParser.usingMinutePrefix(mConfig)).thenReturn("min="); + + timestamp = System.currentTimeMillis(); + } + + private Message buildMessage(long timestamp, int timestampTwo, long timestampThree) throws Exception { + UnitTestMessage thriftMessage = new UnitTestMessage(timestamp, "notimportant", timestampTwo, timestampThree); + TSerializer serializer = new TSerializer(new TBinaryProtocol.Factory()); + byte[] data = serializer.serialize(thriftMessage); + + return new Message("test", 0, 0, null, data, timestamp); + } + + @Test + public void testExtractTimestampFromKafkaTimestamp() throws Exception { + Mockito.when(mConfig.getBoolean("kafka.useTimestamp", false)).thenReturn(true); + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("blasdjlkjasdkl"); + Mockito.when(mConfig.getMessageTimestampId()).thenReturn(1); + Mockito.when(mConfig.getMessageTimestampType()).thenReturn("i64"); + Mockito.when(mConfig.getThriftProtocolClass()).thenReturn("org.apache.thrift.protocol.TBinaryProtocol"); + + ThriftMessageParser parser = new ThriftMessageParser(mConfig); + + assertEquals(1405970352000L, parser.extractTimestampMillis(buildMessage(1405970352L, 1, 2L))); + assertEquals(1405970352123L, parser.extractTimestampMillis(buildMessage(1405970352123L, 1, 2L))); + } + + @Test + public void testExtractTimestamp() throws Exception { + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("blasdjlkjasdkl"); + Mockito.when(mConfig.getMessageTimestampId()).thenReturn(1); + Mockito.when(mConfig.getMessageTimestampType()).thenReturn("i64"); + Mockito.when(mConfig.getThriftProtocolClass()).thenReturn("org.apache.thrift.protocol.TBinaryProtocol"); + + ThriftMessageParser parser = new ThriftMessageParser(mConfig); + + assertEquals(1405970352000L, parser.extractTimestampMillis(buildMessage(1405970352L, 1, 2L))); + assertEquals(1405970352123L, parser.extractTimestampMillis(buildMessage(1405970352123L, 1, 2L))); + } + + @Test + public void testExtractTimestampTwo() throws Exception { + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestampTwo"); + Mockito.when(mConfig.getMessageTimestampId()).thenReturn(3); + Mockito.when(mConfig.getMessageTimestampType()).thenReturn("i32"); + Mockito.when(mConfig.getThriftProtocolClass()).thenReturn("org.apache.thrift.protocol.TBinaryProtocol"); + + ThriftMessageParser parser = new ThriftMessageParser(mConfig); + + assertEquals(1405970352000L, parser.extractTimestampMillis(buildMessage(1L, 1405970352, 2L))); + assertEquals(145028289000L, parser.extractTimestampMillis(buildMessage(1L, 145028289, 2L))); + } + + @Test + public void testExtractTimestampThree() throws Exception { + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("timestampThree"); + Mockito.when(mConfig.getMessageTimestampId()).thenReturn(6); + Mockito.when(mConfig.getMessageTimestampType()).thenReturn("i64"); + Mockito.when(mConfig.getThriftProtocolClass()).thenReturn("org.apache.thrift.protocol.TBinaryProtocol"); + + ThriftMessageParser parser = new ThriftMessageParser(mConfig); + + assertEquals(1405970352000L, parser.extractTimestampMillis(buildMessage(1L, 2, 1405970352L))); + assertEquals(1405970352123L, parser.extractTimestampMillis(buildMessage(1L, 2, 1405970352123L))); + } + + @Test(expected = NullPointerException.class) + public void testAttemptExtractInvalidField() throws Exception { + Mockito.when(mConfig.getMessageTimestampName()).thenReturn("requiredField"); + Mockito.when(mConfig.getMessageTimestampId()).thenReturn(2); + Mockito.when(mConfig.getMessageTimestampType()).thenReturn("i64"); + + ThriftMessageParser parser = new ThriftMessageParser(mConfig); + + parser.extractTimestampMillis(buildMessage(1L, 2, 3L)); + } +} diff --git a/src/test/java/com/pinterest/secor/performance/PerformanceTest010.java b/src/test/java/com/pinterest/secor/performance/PerformanceTest010.java new file mode 100644 index 000000000..dd10ae14c --- /dev/null +++ b/src/test/java/com/pinterest/secor/performance/PerformanceTest010.java @@ -0,0 +1,278 @@ +///** +// * 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. +// */ +// Uncomment below to have this run using Kafka 0.10 +//package com.pinterest.secor.performance; +// +//import java.util.Arrays; +//import java.util.HashMap; +//import java.util.LinkedList; +//import java.util.List; +//import java.util.Map; +//import java.util.Properties; +//import java.util.Random; +// +//import kafka.admin.RackAwareMode; +//import kafka.utils.ZkUtils; +//import org.I0Itec.zkclient.ZkClient; +// +//import com.google.common.collect.Lists; +//import com.google.common.collect.Maps; +//import com.pinterest.secor.common.KafkaClient; +//import com.pinterest.secor.common.OffsetTracker; +//import com.pinterest.secor.common.OstrichAdminService; +//import com.pinterest.secor.common.SecorConfig; +//import com.pinterest.secor.common.TopicPartition; +//import com.pinterest.secor.consumer.Consumer; +//import com.pinterest.secor.tools.LogFileDeleter; +//import com.pinterest.secor.util.FileUtil; +//import com.pinterest.secor.util.RateLimitUtil; +// +//import kafka.admin.AdminUtils; +//import kafka.api.PartitionOffsetRequestInfo; +//import kafka.common.TopicAndPartition; +//import kafka.common.TopicExistsException; +//import kafka.javaapi.OffsetResponse; +//import kafka.javaapi.consumer.SimpleConsumer; +//import kafka.javaapi.producer.Producer; +//import kafka.producer.KeyedMessage; +//import kafka.producer.ProducerConfig; +//import kafka.utils.ZKStringSerializer$; +//import org.I0Itec.zkclient.ZkConnection; +// +///** +// * A performance test for secor +// * +// * * Run: $ cd optimus/secor $ mvn package $ cd target $ java -ea +// * -Dlog4j.configuration=log4j.dev.properties +// * -Dconfig=secor.test.perf.backup.properties \ -cp +// * "secor-0.1-SNAPSHOT-tests.jar:lib/*:secor-0.1-SNAPSHOT.jar" +// * com.pinterest.secor.performance.PerformanceTest +// * +// * +// * @author Praveen Murugesan(praveen@uber.com) +// * +// */ +//public class PerformanceTest010 { +// +// public static void main(String[] args) throws Exception { +// if (args.length != 4) { +// System.err.println("USAGE: java " + PerformanceTest010.class.getName() +// + " num_topics num_partitions num_records message_size"); +// System.exit(1); +// } +// Random rnd = new Random(); +// int num_topics = Integer.parseInt(args[0]); +// SecorConfig config = SecorConfig.load(); +// String zkConfig = config.getZookeeperQuorum() +// + config.getKafkaZookeeperPath(); +// // create topics list +// String perfTopicPrefix = config.getPerfTestTopicPrefix(); +// List topics = Lists.newLinkedList(); +// for (int i = 0; i < num_topics; i++) { +// topics.add(perfTopicPrefix + rnd.nextInt(9999)); +// } +// +// int num_partitions = Integer.parseInt(args[1]); +// +// // createTopics +// createTopics(topics, num_partitions, zkConfig); +// +// int numRecords = Integer.parseInt(args[2]); +// Properties props = new Properties(); +// props.put("metadata.broker.list", "localhost:9092"); +// props.put("serializer.class", "kafka.serializer.StringEncoder"); +// props.put("request.required.acks", "1"); +// props.put("producer.type", "async"); +// +// ProducerConfig producerConfig = new ProducerConfig(props); +// +// Producer producer = new Producer( +// producerConfig); +// long size = 0; +// int message_size = Integer.parseInt(args[3]); +// +// // produce messages +// for (String topic : topics) { +// for (long nEvents = 0; nEvents < numRecords; nEvents++) { +// String ip = String.valueOf(nEvents % num_partitions); +// byte[] payload = new byte[message_size]; +// Arrays.fill(payload, (byte) 1); +// String msg = new String(payload, "UTF-8"); +// size += msg.length(); +// KeyedMessage data = new KeyedMessage( +// topic, ip, msg); +// producer.send(data); +// } +// } +// producer.close(); +// +// RateLimitUtil.configure(config); +// Map lastOffsets = getTopicMetadata(topics, +// num_partitions, config); +// OstrichAdminService ostrichService = new OstrichAdminService( +// config.getOstrichPort()); +// ostrichService.start(); +// FileUtil.configure(config); +// +// LogFileDeleter logFileDeleter = new LogFileDeleter(config); +// logFileDeleter.deleteOldLogs(); +// Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler() { +// public void uncaughtException(Thread thread, Throwable exception) { +// exception.printStackTrace(); +// System.out.println("Thread " + thread + " failed:" +// + exception.getMessage()); +// System.exit(1); +// } +// }; +// System.out.println("starting " + config.getConsumerThreads() +// + " consumer threads"); +// System.out.println("Rate limit:" + config.getMessagesPerSecond()); +// LinkedList consumers = new LinkedList(); +// long startMillis = System.currentTimeMillis(); +// for (int i = 0; i < config.getConsumerThreads(); ++i) { +// Consumer consumer = new Consumer(config); +// consumer.setUncaughtExceptionHandler(handler); +// consumers.add(consumer); +// consumer.start(); +// } +// +// while (true) { +// for (Consumer consumer : consumers) { +// for (String topic : topics) { +// for (int i = 0; i < num_partitions; i++) { +// OffsetTracker offsetTracker = consumer +// .getOffsetTracker(); +// long val = (offsetTracker == null) ? -1 +// : offsetTracker +// .getLastSeenOffset(new TopicPartition( +// topic, i)) + 1; +// +// System.out.println("topic:" + topic + " partition:" + i +// + " secor offset:" + val + " elapsed:" +// + (System.currentTimeMillis() - startMillis)); +// Long lastOffset = lastOffsets.get(new TopicPartition( +// topic, i)); +// if (lastOffset != null && lastOffset == val) { +// lastOffsets.remove(new TopicPartition(topic, i)); +// } +// } +// } +// } +// +// // time break to measure +// Thread.sleep(1000); +// System.out.println("last offsets size:" + lastOffsets.size()); +// if (lastOffsets.isEmpty()) { +// long endMillis = System.currentTimeMillis(); +// System.out.println("Completed in:" + (endMillis - startMillis)); +// System.out.println("Total bytes:" + size); +// // wait for the last file to be written +// Thread.sleep(60000); +// break; +// } +// } +// +// System.exit(1); +// } +// +// /** +// * Get topic partition to last offset map +// * +// * @param topics +// * @param num_partitions +// * @param config +// * @return +// */ +// private static Map getTopicMetadata( +// List topics, int num_partitions, SecorConfig config) { +// KafkaClient mKafkaClient = new KafkaClient(config); +// +// Map lastOffsets = Maps.newHashMap(); +// for (String topic : topics) { +// for (int i = 0; i < num_partitions; i++) { +// TopicAndPartition topicAndPartition = new TopicAndPartition( +// topic, i); +// SimpleConsumer consumer = mKafkaClient +// .createConsumer(new TopicPartition(topic, i)); +// Map requestInfo = new HashMap(); +// requestInfo.put(topicAndPartition, +// new PartitionOffsetRequestInfo(-1, 1)); +// kafka.javaapi.OffsetRequest request = new kafka.javaapi.OffsetRequest( +// requestInfo, kafka.api.OffsetRequest.CurrentVersion(), +// "TestPerformance"); +// OffsetResponse response = consumer.getOffsetsBefore(request); +// if (response.hasError()) { +// System.out +// .println("Error fetching data Offset Data the Broker. Reason: " +// + response.errorCode(topic, i)); +// return null; +// } +// long[] offsets = response.offsets(topic, i); +// System.out.println("Topic: " + topic + " partition: " + i +// + " offset: " + offsets[0]); +// lastOffsets.put(new TopicPartition(topic, i), offsets[0]); +// } +// } +// return lastOffsets; +// } +// +// /** +// * Helper to create topics +// * +// * @param topics +// * @param partitions +// * @param zkConfig +// * @throws InterruptedException +// */ +// private static void createTopics(List topics, int partitions, +// String zkConfig) throws InterruptedException { +// +// ZkConnection zkConnection = new ZkConnection(zkConfig); +// ZkClient zkClient = createZkClient(zkConfig); +// ZkUtils zkUtils = new ZkUtils(zkClient, zkConnection, false); +// +// try { +// Properties props = new Properties(); +// int replicationFactor = 1; +// for (String topic : topics) { +// AdminUtils.createTopic(zkUtils, topic, partitions, +// replicationFactor, props, RackAwareMode.Disabled$.MODULE$); +// } +// } catch (TopicExistsException e) { +// System.out.println(e.getMessage()); +// } finally { +// zkClient.close(); +// } +// +// } +// +// /** +// * Helper to create ZK client +// * +// * @param zkConfig +// * @return +// */ +// private static ZkClient createZkClient(String zkConfig) { +// // Create a ZooKeeper client +// int sessionTimeoutMs = 10000; +// int connectionTimeoutMs = 10000; +// ZkClient zkClient = new ZkClient(zkConfig, sessionTimeoutMs, +// connectionTimeoutMs, ZKStringSerializer$.MODULE$); +// return zkClient; +// } +// +//} \ No newline at end of file diff --git a/src/test/java/com/pinterest/secor/performance/PerformanceTest08.java b/src/test/java/com/pinterest/secor/performance/PerformanceTest08.java new file mode 100644 index 000000000..bd19088e5 --- /dev/null +++ b/src/test/java/com/pinterest/secor/performance/PerformanceTest08.java @@ -0,0 +1,273 @@ +///** +// * 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. +// */ +// Uncomment below to have this run using Kafka 0.8 +//package com.pinterest.secor.performance; +// +//import java.util.Arrays; +//import java.util.HashMap; +//import java.util.LinkedList; +//import java.util.List; +//import java.util.Map; +//import java.util.Properties; +//import java.util.Random; +// +//import org.I0Itec.zkclient.ZkClient; +// +//import com.google.common.collect.Lists; +//import com.google.common.collect.Maps; +//import com.pinterest.secor.common.KafkaClient; +//import com.pinterest.secor.common.OffsetTracker; +//import com.pinterest.secor.common.OstrichAdminService; +//import com.pinterest.secor.common.SecorConfig; +//import com.pinterest.secor.common.TopicPartition; +//import com.pinterest.secor.consumer.Consumer; +//import com.pinterest.secor.tools.LogFileDeleter; +//import com.pinterest.secor.util.FileUtil; +//import com.pinterest.secor.util.RateLimitUtil; +// +//import kafka.admin.AdminUtils; +//import kafka.api.PartitionOffsetRequestInfo; +//import kafka.common.TopicAndPartition; +//import kafka.common.TopicExistsException; +//import kafka.javaapi.OffsetResponse; +//import kafka.javaapi.consumer.SimpleConsumer; +//import kafka.javaapi.producer.Producer; +//import kafka.producer.KeyedMessage; +//import kafka.producer.ProducerConfig; +//import kafka.utils.ZKStringSerializer$; +// +///** +// * A performance test for secor +// * +// * * Run: $ cd optimus/secor $ mvn package $ cd target $ java -ea +// * -Dlog4j.configuration=log4j.dev.properties +// * -Dconfig=secor.test.perf.backup.properties \ -cp +// * "secor-0.1-SNAPSHOT-tests.jar:lib/*:secor-0.1-SNAPSHOT.jar" +// * com.pinterest.secor.performance.PerformanceTest +// * +// * +// * @author Praveen Murugesan(praveen@uber.com) +// * +// */ +//public class PerformanceTest08 { +// +// public static void main(String[] args) throws Exception { +// if (args.length != 4) { +// System.err.println("USAGE: java " + PerformanceTest010.class.getName() +// + " num_topics num_partitions num_records message_size"); +// System.exit(1); +// } +// Random rnd = new Random(); +// int num_topics = Integer.parseInt(args[0]); +// SecorConfig config = SecorConfig.load(); +// String zkConfig = config.getZookeeperQuorum() +// + config.getKafkaZookeeperPath(); +// // create topics list +// String perfTopicPrefix = config.getPerfTestTopicPrefix(); +// List topics = Lists.newLinkedList(); +// for (int i = 0; i < num_topics; i++) { +// topics.add(perfTopicPrefix + rnd.nextInt(9999)); +// } +// +// int num_partitions = Integer.parseInt(args[1]); +// +// // createTopics +// createTopics(topics, num_partitions, zkConfig); +// +// int numRecords = Integer.parseInt(args[2]); +// Properties props = new Properties(); +// props.put("metadata.broker.list", "localhost:9092"); +// props.put("serializer.class", "kafka.serializer.StringEncoder"); +// props.put("request.required.acks", "1"); +// props.put("producer.type", "async"); +// +// ProducerConfig producerConfig = new ProducerConfig(props); +// +// Producer producer = new Producer( +// producerConfig); +// long size = 0; +// int message_size = Integer.parseInt(args[3]); +// +// // produce messages +// for (String topic : topics) { +// for (long nEvents = 0; nEvents < numRecords; nEvents++) { +// String ip = String.valueOf(nEvents % num_partitions); +// byte[] payload = new byte[message_size]; +// Arrays.fill(payload, (byte) 1); +// String msg = new String(payload, "UTF-8"); +// size += msg.length(); +// KeyedMessage data = new KeyedMessage( +// topic, ip, msg); +// producer.send(data); +// } +// } +// producer.close(); +// +// RateLimitUtil.configure(config); +// Map lastOffsets = getTopicMetadata(topics, +// num_partitions, config); +// OstrichAdminService ostrichService = new OstrichAdminService( +// config.getOstrichPort()); +// ostrichService.start(); +// FileUtil.configure(config); +// +// LogFileDeleter logFileDeleter = new LogFileDeleter(config); +// logFileDeleter.deleteOldLogs(); +// Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler() { +// public void uncaughtException(Thread thread, Throwable exception) { +// exception.printStackTrace(); +// System.out.println("Thread " + thread + " failed:" +// + exception.getMessage()); +// System.exit(1); +// } +// }; +// System.out.println("starting " + config.getConsumerThreads() +// + " consumer threads"); +// System.out.println("Rate limit:" + config.getMessagesPerSecond()); +// LinkedList consumers = new LinkedList(); +// long startMillis = System.currentTimeMillis(); +// for (int i = 0; i < config.getConsumerThreads(); ++i) { +// Consumer consumer = new Consumer(config); +// consumer.setUncaughtExceptionHandler(handler); +// consumers.add(consumer); +// consumer.start(); +// } +// +// while (true) { +// for (Consumer consumer : consumers) { +// for (String topic : topics) { +// for (int i = 0; i < num_partitions; i++) { +// OffsetTracker offsetTracker = consumer +// .getOffsetTracker(); +// long val = (offsetTracker == null) ? -1 +// : offsetTracker +// .getLastSeenOffset(new TopicPartition( +// topic, i)) + 1; +// +// System.out.println("topic:" + topic + " partition:" + i +// + " secor offset:" + val + " elapsed:" +// + (System.currentTimeMillis() - startMillis)); +// Long lastOffset = lastOffsets.get(new TopicPartition( +// topic, i)); +// if (lastOffset != null && lastOffset == val) { +// lastOffsets.remove(new TopicPartition(topic, i)); +// } +// } +// } +// } +// +// // time break to measure +// Thread.sleep(1000); +// System.out.println("last offsets size:" + lastOffsets.size()); +// if (lastOffsets.isEmpty()) { +// long endMillis = System.currentTimeMillis(); +// System.out.println("Completed in:" + (endMillis - startMillis)); +// System.out.println("Total bytes:" + size); +// // wait for the last file to be written +// Thread.sleep(60000); +// break; +// } +// } +// +// System.exit(1); +// } +// +// /** +// * Get topic partition to last offset map +// * +// * @param topics +// * @param num_partitions +// * @param config +// * @return +// */ +// private static Map getTopicMetadata( +// List topics, int num_partitions, SecorConfig config) { +// KafkaClient mKafkaClient = new KafkaClient(config); +// +// Map lastOffsets = Maps.newHashMap(); +// for (String topic : topics) { +// for (int i = 0; i < num_partitions; i++) { +// TopicAndPartition topicAndPartition = new TopicAndPartition( +// topic, i); +// SimpleConsumer consumer = mKafkaClient +// .createConsumer(new TopicPartition(topic, i)); +// Map requestInfo = new HashMap(); +// requestInfo.put(topicAndPartition, +// new PartitionOffsetRequestInfo(-1, 1)); +// kafka.javaapi.OffsetRequest request = new kafka.javaapi.OffsetRequest( +// requestInfo, kafka.api.OffsetRequest.CurrentVersion(), +// "TestPerformance"); +// OffsetResponse response = consumer.getOffsetsBefore(request); +// if (response.hasError()) { +// System.out +// .println("Error fetching data Offset Data the Broker. Reason: " +// + response.errorCode(topic, i)); +// return null; +// } +// long[] offsets = response.offsets(topic, i); +// System.out.println("Topic: " + topic + " partition: " + i +// + " offset: " + offsets[0]); +// lastOffsets.put(new TopicPartition(topic, i), offsets[0]); +// } +// } +// return lastOffsets; +// } +// +// /** +// * Helper to create topics +// * +// * @param topics +// * @param partitions +// * @param zkConfig +// * @throws InterruptedException +// */ +// private static void createTopics(List topics, int partitions, +// String zkConfig) throws InterruptedException { +// +// ZkClient zkClient = createZkClient(zkConfig); +// +// try { +// Properties props = new Properties(); +// int replicationFactor = 1; +// for (String topic : topics) { +// AdminUtils.createTopic(zkClient, topic, partitions, +// replicationFactor, props); +// } +// } catch (TopicExistsException e) { +// System.out.println(e.getMessage()); +// } finally { +// zkClient.close(); +// } +// +// } +// +// /** +// * Helper to create ZK client +// * +// * @param zkConfig +// * @return +// */ +// private static ZkClient createZkClient(String zkConfig) { +// // Create a ZooKeeper client +// int sessionTimeoutMs = 10000; +// int connectionTimeoutMs = 10000; +// ZkClient zkClient = new ZkClient(zkConfig, sessionTimeoutMs, +// connectionTimeoutMs, ZKStringSerializer$.MODULE$); +// return zkClient; +// } +// +//} \ No newline at end of file diff --git a/src/test/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactoryTest.java b/src/test/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactoryTest.java new file mode 100644 index 000000000..1dec6cd76 --- /dev/null +++ b/src/test/java/com/pinterest/secor/timestamp/KafkaMessageTimestampFactoryTest.java @@ -0,0 +1,23 @@ +package com.pinterest.secor.timestamp; + +import org.junit.Test; + +import static org.junit.Assert.*; + +public class KafkaMessageTimestampFactoryTest { + + private KafkaMessageTimestampFactory factory; + + @Test + public void shouldReturnKafka8TimestampClassObject() { + factory = new KafkaMessageTimestampFactory("com.pinterest.secor.timestamp.Kafka8MessageTimestamp"); + Object timestamp = factory.getKafkaMessageTimestamp(); + assertNotNull(timestamp); + assertEquals(timestamp.getClass(), Kafka8MessageTimestamp.class); + } + + @Test(expected = RuntimeException.class) + public void shouldReturnNullForInvalidClass() { + factory = new KafkaMessageTimestampFactory("com.pinterest.secor.timestamp.KafkaxxMessageTimestamp"); + } +} diff --git a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java index cd96f8a3f..335afee9f 100644 --- a/src/test/java/com/pinterest/secor/uploader/UploaderTest.java +++ b/src/test/java/com/pinterest/secor/uploader/UploaderTest.java @@ -17,15 +17,21 @@ package com.pinterest.secor.uploader; import com.pinterest.secor.common.*; +import com.pinterest.secor.io.FileReader; +import com.pinterest.secor.io.FileWriter; +import com.pinterest.secor.io.KeyValue; +import com.pinterest.secor.monitoring.MetricCollector; 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.apache.hadoop.io.compress.CompressionCodec; +import org.joda.time.DateTime; import org.junit.runner.RunWith; +import org.mockito.Mock; import org.mockito.Mockito; +import org.mockito.internal.exceptions.ExceptionIncludingMockitoWarnings; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.powermock.api.mockito.PowerMockito; @@ -33,7 +39,9 @@ import org.powermock.modules.junit4.PowerMockRunner; import java.io.IOException; +import java.util.HashMap; import java.util.HashSet; +import java.util.Map; /** * UploaderTest tests the log file uploader logic. @@ -41,24 +49,26 @@ * @author Pawel Garbacki (pawel@pinterest.com) */ @RunWith(PowerMockRunner.class) -@PrepareForTest({FileUtil.class, FileSystem.class, IdUtil.class}) +@PrepareForTest({ FileUtil.class, IdUtil.class , DateTime.class}) public class UploaderTest extends TestCase { private static class TestUploader extends Uploader { - private SequenceFile.Reader mReader; + private FileReader mReader; public TestUploader(SecorConfig config, OffsetTracker offsetTracker, - FileRegistry fileRegistry, ZookeeperConnector zookeeperConnector) { - super(config, offsetTracker, fileRegistry, zookeeperConnector); - mReader = Mockito.mock(SequenceFile.Reader.class); + FileRegistry fileRegistry, + UploadManager uploadManager, + ZookeeperConnector zookeeperConnector) { + init(config, offsetTracker, fileRegistry, uploadManager, zookeeperConnector, Mockito.mock(MetricCollector.class)); + mReader = Mockito.mock(FileReader.class); } @Override - protected SequenceFile.Reader createReader(FileSystem fileSystem, Path path, - Configuration configuration) throws IOException { + protected FileReader createReader(LogFilePath srcPath, + CompressionCodec codec) throws IOException { return mReader; } - public SequenceFile.Reader getReader() { + public FileReader getReader() { return mReader; } } @@ -71,6 +81,7 @@ public SequenceFile.Reader getReader() { private OffsetTracker mOffsetTracker; private FileRegistry mFileRegistry; private ZookeeperConnector mZookeeperConnector; + private UploadManager mUploadManager; private TestUploader mUploader; @@ -80,12 +91,13 @@ public void setUp() throws Exception { 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"); + "/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); + Mockito.when(mConfig.getZookeeperPath()).thenReturn("/"); mOffsetTracker = Mockito.mock(OffsetTracker.class); @@ -93,47 +105,160 @@ public void setUp() throws Exception { Mockito.when(mFileRegistry.getSize(mTopicPartition)).thenReturn(100L); HashSet topicPartitions = new HashSet(); topicPartitions.add(mTopicPartition); - Mockito.when(mFileRegistry.getTopicPartitions()).thenReturn(topicPartitions); + Mockito.when(mFileRegistry.getTopicPartitions()).thenReturn( + topicPartitions); + + mUploadManager = new HadoopS3UploadManager(mConfig); mZookeeperConnector = Mockito.mock(ZookeeperConnector.class); - mUploader = new TestUploader(mConfig, mOffsetTracker, mFileRegistry, mZookeeperConnector); + mUploader = new TestUploader(mConfig, mOffsetTracker, mFileRegistry, mUploadManager, + mZookeeperConnector); + } + + public void testUploadAtTime() throws Exception { + final int minuteUploadMark = 1; + + PowerMockito.mockStatic(DateTime.class); + PowerMockito.when(DateTime.now()).thenReturn(new DateTime(2016,7,27,0,minuteUploadMark,0)); + Mockito.when(mConfig.getUploadMinuteMark()).thenReturn(minuteUploadMark); + Mockito.when(mConfig.getKafkaTopicFilter()).thenReturn("some_topic"); + + Mockito.when(mConfig.getCloudService()).thenReturn("S3"); + 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); + Mockito.when(FileUtil.getPrefix("some_topic", mConfig)). + thenReturn("s3a://some_bucket/some_s3_parent_dir"); + mUploader.applyPolicy(); + + final String lockPath = "/secor/locks/some_topic/0"; + Mockito.verify(mZookeeperConnector).lock(lockPath); + PowerMockito.verifyStatic(); + FileUtil.moveToCloud( + "/some_parent_dir/some_topic/some_partition/some_other_partition/" + + "10_0_00000000000000000010", + "s3a://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, 1L); + Mockito.verify(mOffsetTracker).setCommittedOffsetCount(mTopicPartition, + 1L); + Mockito.verify(mZookeeperConnector).unlock(lockPath); } 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( + 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.getCloudService()).thenReturn("S3"); 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); + Mockito.when(mFileRegistry.getPaths(mTopicPartition)).thenReturn( + logFilePaths); PowerMockito.mockStatic(FileUtil.class); + Mockito.when(FileUtil.getPrefix("some_topic", mConfig)). + thenReturn("s3a://some_bucket/some_s3_parent_dir"); + mUploader.applyPolicy(); + + final String lockPath = "/secor/locks/some_topic/0"; + Mockito.verify(mZookeeperConnector).lock(lockPath); + PowerMockito.verifyStatic(); + FileUtil.moveToCloud( + "/some_parent_dir/some_topic/some_partition/some_other_partition/" + + "10_0_00000000000000000010", + "s3a://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 testUploadFilesCustomTopicName() 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.getCloudService()).thenReturn("S3"); + Mockito.when(mConfig.getS3Bucket()).thenReturn("some_bucket"); + Mockito.when(mConfig.getS3Path()).thenReturn("some_s3_parent_dir"); + Map customTopicsNamesMap = new HashMap(); + customTopicsNamesMap.put("some_topic", "some_topic_test"); + Mockito.when(mConfig.getCustomTopicsNames()).thenReturn(customTopicsNamesMap); + HashSet logFilePaths = new HashSet(); + logFilePaths.add(mLogFilePath); + Mockito.when(mFileRegistry.getPaths(mTopicPartition)).thenReturn( + logFilePaths); + + PowerMockito.mockStatic(FileUtil.class); + Mockito.when(FileUtil.getPrefix("some_topic", mConfig)). + thenReturn("s3a://some_bucket/some_s3_parent_dir"); 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"); + FileUtil.moveToCloud( + "/some_parent_dir/some_topic/some_partition/some_other_partition/" + + "10_0_00000000000000000010", + "s3a://some_bucket/some_s3_parent_dir/some_topic_test/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).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); + Mockito.when( + mZookeeperConnector.getCommittedOffsetCount(mTopicPartition)) + .thenReturn(31L); + Mockito.when( + mOffsetTracker.setCommittedOffsetCount(mTopicPartition, 30L)) + .thenReturn(11L); + Mockito.when(mOffsetTracker.getLastSeenOffset(mTopicPartition)) + .thenReturn(20L); mUploader.applyPolicy(); @@ -141,47 +266,50 @@ public void testDeleteTopicPartition() throws Exception { } 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); + 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); + Mockito.when(mFileRegistry.getPaths(mTopicPartition)).thenReturn( + logFilePaths); - PowerMockito.mockStatic(FileSystem.class); + FileReader reader = mUploader.getReader(); - 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() { + Mockito.when(reader.next()).thenAnswer(new Answer() { private int mCallCount = 0; + @Override - public Boolean answer(InvocationOnMock invocation) throws Throwable { + public KeyValue answer(InvocationOnMock invocation) + throws Throwable { if (mCallCount == 2) { - return false; + return null; } - LongWritable key = (LongWritable) invocation.getArguments()[0]; - key.set(20 + mCallCount++); - return true; + return new KeyValue(20 + mCallCount++, null); } }); PowerMockito.mockStatic(IdUtil.class); - Mockito.when(IdUtil.getLocalMessageDir()).thenReturn("some_message_dir"); + 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, null)).thenReturn(writer); + FileWriter writer = Mockito.mock(FileWriter.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, null)) + .thenReturn(writer); mUploader.applyPolicy(); - Mockito.verify(writer).append(Mockito.any(LongWritable.class), - Mockito.any(BytesWritable.class)); + Mockito.verify(writer).write(Mockito.any(KeyValue.class)); Mockito.verify(mFileRegistry).deletePath(mLogFilePath); } } diff --git a/src/test/java/com/pinterest/secor/util/FileUtilTest.java b/src/test/java/com/pinterest/secor/util/FileUtilTest.java new file mode 100644 index 000000000..185eb300b --- /dev/null +++ b/src/test/java/com/pinterest/secor/util/FileUtilTest.java @@ -0,0 +1,82 @@ +/** + * 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 org.mockito.Mockito; +import org.junit.Test; +import org.junit.Assert; +import org.junit.Before; + +import com.pinterest.secor.common.SecorConfig; + +public class FileUtilTest { + + private SecorConfig mSwiftConfig; + private SecorConfig mS3AConfig; + private SecorConfig mS3NConfig; + private SecorConfig mGSconfig; + + @Before + public void setUp() throws Exception { + mSwiftConfig = Mockito.mock(SecorConfig.class); + Mockito.when(mSwiftConfig.getCloudService()).thenReturn("Swift"); + Mockito.when(mSwiftConfig.getSeparateContainersForTopics()).thenReturn(false); + Mockito.when(mSwiftConfig.getSwiftContainer()).thenReturn("some_container"); + Mockito.when(mSwiftConfig.getSwiftPath()).thenReturn("some_swift_parent_dir"); + + mS3AConfig = Mockito.mock(SecorConfig.class); + Mockito.when(mS3AConfig.getCloudService()).thenReturn("S3"); + Mockito.when(mS3AConfig.getS3FileSystem()).thenReturn("s3a"); + Mockito.when(mS3AConfig.getS3Bucket()).thenReturn("some_bucket"); + Mockito.when(mS3AConfig.getS3Path()).thenReturn("some_s3_parent_dir"); + Mockito.when(mS3AConfig.getS3Prefix()).thenCallRealMethod(); + + mS3NConfig = Mockito.mock(SecorConfig.class); + Mockito.when(mS3NConfig.getCloudService()).thenReturn("S3"); + Mockito.when(mS3NConfig.getS3FileSystem()).thenReturn("s3n"); + Mockito.when(mS3NConfig.getS3Bucket()).thenReturn("some_bucket"); + Mockito.when(mS3NConfig.getS3Path()).thenReturn("some_s3_parent_dir"); + Mockito.when(mS3NConfig.getS3Prefix()).thenCallRealMethod(); + + mGSconfig = Mockito.mock(SecorConfig.class); + Mockito.when(mGSconfig.getCloudService()).thenReturn("GS"); + Mockito.when(mGSconfig.getGsBucket()).thenReturn("some_gs_bucket"); + Mockito.when(mGSconfig.getGsPath()).thenReturn("some_gs_parent_dir"); + } + + @Test + public void testGetPrefix() throws Exception { + //FileUtil.configure(mSwiftConfig); + Assert.assertEquals(FileUtil.getPrefix("some_topic", mSwiftConfig), + "swift://some_container.GENERICPROJECT/some_swift_parent_dir"); + + //FileUtil.configure(mS3AConfig); + Assert.assertEquals(FileUtil.getPrefix("some_topic", mS3AConfig), + "s3a://some_bucket/some_s3_parent_dir"); + + //FileUtil.configure(mS3NConfig); + Assert.assertEquals(FileUtil.getPrefix("some_topic", mS3NConfig), + "s3n://some_bucket/some_s3_parent_dir"); + + //FileUtil.configure(mGSConfig); + Assert.assertEquals(FileUtil.getPrefix("some_topic", mGSconfig), + "gs://some_gs_bucket/some_gs_parent_dir"); + + // return to the previous state + FileUtil.configure(null); + } +} diff --git a/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java b/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java new file mode 100644 index 000000000..8ad8c2b4e --- /dev/null +++ b/src/test/java/com/pinterest/secor/util/ReflectionUtilTest.java @@ -0,0 +1,97 @@ +/** + * 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.LogFilePath; +import com.pinterest.secor.common.SecorConfig; +import com.pinterest.secor.monitoring.MetricCollector; +import com.pinterest.secor.monitoring.OstrichMetricCollector; +import com.pinterest.secor.parser.MessageParser; +import org.apache.commons.configuration.PropertiesConfiguration; +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class ReflectionUtilTest { + + private SecorConfig mSecorConfig; + private LogFilePath mLogFilePath; + + @Before + public void setUp() throws Exception { + PropertiesConfiguration properties = new PropertiesConfiguration(); + properties.addProperty("message.timestamp.name", ""); + properties.addProperty("message.timestamp.name.separator", ""); + properties.addProperty("secor.offsets.prefix", "offset="); + mSecorConfig = new SecorConfig(properties); + mLogFilePath = new LogFilePath("/foo", "/foo/bar/baz/1_1_1"); + + } + + @Test + public void testCreateMessageParser() throws Exception { + MessageParser messageParser = ReflectionUtil.createMessageParser("com.pinterest.secor.parser.OffsetMessageParser", + mSecorConfig); + } + + @Test(expected = ClassNotFoundException.class) + public void testMessageParserClassNotFound() throws Exception { + ReflectionUtil.createMessageParser("com.example.foo", mSecorConfig); + } + + @Test(expected = ClassNotFoundException.class) + public void testFileWriterClassNotFound() throws Exception { + ReflectionUtil.createFileWriter("com.example.foo", mLogFilePath, null, mSecorConfig); + } + + @Test(expected = IllegalArgumentException.class) + public void testMessageParserConstructorMissing() throws Exception { + // Try to create a message parser using an existent and available class, but one not + // assignable to MessageParser + ReflectionUtil.createMessageParser("java.lang.Object", + mSecorConfig); + } + + @Test(expected = IllegalArgumentException.class) + public void testFileWriterConstructorMissing() throws Exception { + // Try to create a message parser using an existent and available class, but one not + // assignable to MessageParser + ReflectionUtil.createFileWriter("java.lang.Object", + mLogFilePath, null, mSecorConfig); + } + + @Test + public void testCreateMetricsCollector() throws Exception { + MetricCollector metricCollector = ReflectionUtil.createMetricCollector("com.pinterest.secor.monitoring.OstrichMetricCollector"); + + Assert.assertNotNull(metricCollector); + Assert.assertThat(metricCollector, CoreMatchers.instanceOf(OstrichMetricCollector.class)); + } + + @Test(expected = ClassNotFoundException.class) + public void testCreateMetricsCollectorClassNotFound() throws Exception { + ReflectionUtil.createMetricCollector("com.example.foo"); + } + + @Test(expected = IllegalArgumentException.class) + public void testCreateMetricsCollectorNotAssignable() throws Exception { + // Try to create a message parser using an existent and available class, but one not + // assignable to MessageParser + ReflectionUtil.createMetricCollector("java.lang.Object"); + } +} diff --git a/src/test/protobuf/unittest.proto b/src/test/protobuf/unittest.proto new file mode 100644 index 000000000..c06f8266d --- /dev/null +++ b/src/test/protobuf/unittest.proto @@ -0,0 +1,23 @@ +syntax = "proto2"; + +package com.pinterest.secor.protobuf; +option java_package = "com.pinterest.secor.protobuf"; +option java_outer_classname = "Messages"; + +message UnitTestMessage1 { + optional string somefield = 1; + optional uint64 timestamp = 2; +} + +message UnitTestMessage2 { + message Internal { + optional uint64 timestamp = 1; + } + + optional Internal internal = 1; +} + +message UnitTestMessage3 { + optional string data = 1; + optional uint64 timestamp = 2; +} \ No newline at end of file diff --git a/src/test/protobuf/unittesttimestamp.proto b/src/test/protobuf/unittesttimestamp.proto new file mode 100644 index 000000000..1aa277172 --- /dev/null +++ b/src/test/protobuf/unittesttimestamp.proto @@ -0,0 +1,19 @@ +syntax = "proto3"; +package com.pinterest.secor.protobuf; + +option java_package = "com.pinterest.secor.protobuf"; +import "google/protobuf/timestamp.proto"; +option java_outer_classname = "TimestampedMessages"; + + +message UnitTestTimestamp1 { + google.protobuf.Timestamp timestamp = 5; +} + +message UnitTestTimestamp2 { + message Internal { + google.protobuf.Timestamp timestamp = 1; + } + + Internal internal = 1; +} \ No newline at end of file diff --git a/src/test/thrift/unittest.thrift b/src/test/thrift/unittest.thrift new file mode 100644 index 000000000..cbc29fbab --- /dev/null +++ b/src/test/thrift/unittest.thrift @@ -0,0 +1,18 @@ +/* Simple thrift message used in Secor unit testing */ + +namespace java com.pinterest.secor.thrift + +enum UnitTestEnum { + SOME_VALUE = 0, + SOME_OTHER_VALUE = 1, +} + +struct UnitTestMessage { + 1: required i64 timestamp, + 2: required string requiredField, + 3: required i32 timestampTwo, + 4: optional string optionalField, + 5: optional UnitTestEnum enumField, + 6: required i64 timestampThree +} +